[NO ISSUE][COMP] Make LeftOuterJoin produce NULL in SQL-compat mode

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

Details:
- SQL-compat mode: make Left Outer Join emit NULL instead of
  MISSING for non-matching values
- SQL-compat mode: make Left Outer Unnest emit NULL instead of
  MISSING if there are no items in the input list
- Fix comparison of AsterixConstantValue with
  IAlgebricksConstantValue which is NULL/MISSING or BOOLEAN
- Add testcases

Change-Id: Ib39bd4e704015bbd38a2bf0b8d3a54bdbd7cd7e1
Reviewed-on: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/14244
Integration-Tests: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Ali Alsuliman <ali.al.solaiman@gmail.com>
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java
index ed8e9bf..06a6687 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java
@@ -21,6 +21,8 @@
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.metadata.declared.DataSourceId;
 import org.apache.asterix.metadata.declared.DataSourceIndex;
 import org.apache.asterix.metadata.declared.MetadataProvider;
@@ -37,7 +39,6 @@
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
-import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
@@ -46,6 +47,7 @@
 import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSourceIndex;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractUnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterUnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.properties.BroadcastPartitioningProperty;
@@ -61,6 +63,7 @@
 import org.apache.hyracks.algebricks.core.algebra.properties.UnorderedPartitionedProperty;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
+import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
 import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
 
 /**
@@ -116,31 +119,45 @@
         int[] lowKeyIndexes = getKeyIndexes(jobGenParams.getLowKeyVarList(), inputSchemas);
         int[] highKeyIndexes = getKeyIndexes(jobGenParams.getHighKeyVarList(), inputSchemas);
 
+        boolean propagateFilter = unnestMap.propagateIndexFilter();
+        IMissingWriterFactory nonFilterWriterFactory = getNonFilterWriterFactory(propagateFilter, context);
         int[] minFilterFieldIndexes = getKeyIndexes(unnestMap.getMinFilterVars(), inputSchemas);
         int[] maxFilterFieldIndexes = getKeyIndexes(unnestMap.getMaxFilterVars(), inputSchemas);
-        boolean propagateFilter = unnestMap.propagateIndexFilter();
 
         MetadataProvider metadataProvider = (MetadataProvider) context.getMetadataProvider();
         Dataset dataset = metadataProvider.findDataset(jobGenParams.getDataverseName(), jobGenParams.getDatasetName());
         IVariableTypeEnvironment typeEnv = context.getTypeEnvironment(op);
         ITupleFilterFactory tupleFilterFactory = null;
         long outputLimit = -1;
-        if (unnestMap.getOperatorTag() == LogicalOperatorTag.UNNEST_MAP) {
-            UnnestMapOperator unnestMapOp = (UnnestMapOperator) unnestMap;
-            outputLimit = unnestMapOp.getOutputLimit();
-            if (unnestMapOp.getSelectCondition() != null) {
-                tupleFilterFactory = metadataProvider.createTupleFilterFactory(new IOperatorSchema[] { opSchema },
-                        typeEnv, unnestMapOp.getSelectCondition().getValue(), context);
-            }
+        boolean retainMissing = false;
+        IMissingWriterFactory nonMatchWriterFactory = null;
+        switch (unnestMap.getOperatorTag()) {
+            case UNNEST_MAP:
+                UnnestMapOperator unnestMapOp = (UnnestMapOperator) unnestMap;
+                outputLimit = unnestMapOp.getOutputLimit();
+                if (unnestMapOp.getSelectCondition() != null) {
+                    tupleFilterFactory = metadataProvider.createTupleFilterFactory(new IOperatorSchema[] { opSchema },
+                            typeEnv, unnestMapOp.getSelectCondition().getValue(), context);
+                }
+                break;
+            case LEFT_OUTER_UNNEST_MAP:
+                // By nature, LEFT_OUTER_UNNEST_MAP should generate missing (or null) values for non-matching tuples.
+                retainMissing = true;
+                nonMatchWriterFactory =
+                        getNonMatchWriterFactory(((LeftOuterUnnestMapOperator) unnestMap).getMissingValue(), context,
+                                unnestMap.getSourceLocation());
+                break;
+            default:
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, unnestMap.getSourceLocation(),
+                        String.valueOf(unnestMap.getOperatorTag()));
         }
-        // By nature, LEFT_OUTER_UNNEST_MAP should generate null values for non-matching tuples.
-        boolean retainMissing = op.getOperatorTag() == LogicalOperatorTag.LEFT_OUTER_UNNEST_MAP;
+
         Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> btreeSearch = metadataProvider.buildBtreeRuntime(
-                builder.getJobSpec(), opSchema, typeEnv, context, jobGenParams.getRetainInput(), retainMissing, dataset,
-                jobGenParams.getIndexName(), lowKeyIndexes, highKeyIndexes, jobGenParams.isLowKeyInclusive(),
-                jobGenParams.isHighKeyInclusive(), propagateFilter, minFilterFieldIndexes, maxFilterFieldIndexes,
-                tupleFilterFactory, outputLimit, unnestMap.getGenerateCallBackProceedResultVar(),
-                isPrimaryIndexPointSearch(op));
+                builder.getJobSpec(), opSchema, typeEnv, context, jobGenParams.getRetainInput(), retainMissing,
+                nonMatchWriterFactory, dataset, jobGenParams.getIndexName(), lowKeyIndexes, highKeyIndexes,
+                jobGenParams.isLowKeyInclusive(), jobGenParams.isHighKeyInclusive(), propagateFilter,
+                nonFilterWriterFactory, minFilterFieldIndexes, maxFilterFieldIndexes, tupleFilterFactory, outputLimit,
+                unnestMap.getGenerateCallBackProceedResultVar(), isPrimaryIndexPointSearch(op));
         IOperatorDescriptor opDesc = btreeSearch.first;
         opDesc.setSourceLocation(unnestMap.getSourceLocation());
 
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/IndexSearchPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/IndexSearchPOperator.java
index 73fcf92..4de30ce 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/IndexSearchPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/IndexSearchPOperator.java
@@ -20,10 +20,13 @@
 
 import java.util.List;
 
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.metadata.declared.DataSourceId;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSource;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSourceIndex;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSourcePropertiesProvider;
@@ -36,6 +39,9 @@
 import org.apache.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
 import org.apache.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
 import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * Class that embodies the commonalities between access method physical operators.
@@ -102,4 +108,21 @@
         }
         return ss;
     }
+
+    protected static IMissingWriterFactory getNonMatchWriterFactory(IAlgebricksConstantValue missingValue,
+            JobGenContext context, SourceLocation sourceLoc) throws CompilationException {
+        IMissingWriterFactory nonMatchWriterFactory;
+        if (missingValue.isMissing()) {
+            nonMatchWriterFactory = context.getMissingWriterFactory();
+        } else if (missingValue.isNull()) {
+            nonMatchWriterFactory = context.getNullWriterFactory();
+        } else {
+            throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc, missingValue.toString());
+        }
+        return nonMatchWriterFactory;
+    }
+
+    protected static IMissingWriterFactory getNonFilterWriterFactory(boolean propagateFilter, JobGenContext context) {
+        return propagateFilter ? context.getMissingWriterFactory() : null;
+    }
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
index 532fb43..00eef69 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
@@ -49,11 +49,13 @@
 import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSourceIndex;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractUnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterUnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
 import org.apache.hyracks.algebricks.core.algebra.properties.LocalMemoryRequirements;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
 import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
+import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
@@ -117,13 +119,18 @@
         Dataset dataset = metadataProvider.findDataset(jobGenParams.getDataverseName(), jobGenParams.getDatasetName());
         int[] keyIndexes = getKeyIndexes(jobGenParams.getKeyVarList(), inputSchemas);
 
+        boolean propagateIndexFilter = unnestMapOp.propagateIndexFilter();
+        IMissingWriterFactory nonFilterWriterFactory = getNonFilterWriterFactory(propagateIndexFilter, context);
         int[] minFilterFieldIndexes = getKeyIndexes(unnestMapOp.getMinFilterVars(), inputSchemas);
         int[] maxFilterFieldIndexes = getKeyIndexes(unnestMapOp.getMaxFilterVars(), inputSchemas);
-        boolean retainNull = false;
+        boolean retainMissing = false;
+        IMissingWriterFactory nonMatchWriterFactory = null;
         if (op.getOperatorTag() == LogicalOperatorTag.LEFT_OUTER_UNNEST_MAP) {
-            // By nature, LEFT_OUTER_UNNEST_MAP should generate null values for non-matching
-            // tuples.
-            retainNull = true;
+            // By nature, LEFT_OUTER_UNNEST_MAP should generate null values for non-matching tuples
+            retainMissing = true;
+            nonMatchWriterFactory =
+                    getNonMatchWriterFactory(((LeftOuterUnnestMapOperator) unnestMapOp).getMissingValue(), context,
+                            unnestMapOp.getSourceLocation());
         }
         // In-memory budget (frame limit) for inverted-index search operations
         int frameLimit = localMemoryRequirements.getMemoryBudgetInFrames();
@@ -131,9 +138,10 @@
         // Build runtime.
         Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> invIndexSearch =
                 buildInvertedIndexRuntime(metadataProvider, context, builder.getJobSpec(), unnestMapOp, opSchema,
-                        jobGenParams.getRetainInput(), retainNull, jobGenParams.getDatasetName(), dataset,
-                        jobGenParams.getIndexName(), jobGenParams.getSearchKeyType(), keyIndexes,
-                        jobGenParams.getSearchModifierType(), jobGenParams.getSimilarityThreshold(),
+                        jobGenParams.getRetainInput(), retainMissing, nonMatchWriterFactory,
+                        jobGenParams.getDatasetName(), dataset, jobGenParams.getIndexName(),
+                        jobGenParams.getSearchKeyType(), keyIndexes, jobGenParams.getSearchModifierType(),
+                        jobGenParams.getSimilarityThreshold(), propagateIndexFilter, nonFilterWriterFactory,
                         minFilterFieldIndexes, maxFilterFieldIndexes, jobGenParams.getIsFullTextSearch(), frameLimit);
         IOperatorDescriptor opDesc = invIndexSearch.first;
         opDesc.setSourceLocation(unnestMapOp.getSourceLocation());
@@ -148,11 +156,11 @@
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildInvertedIndexRuntime(
             MetadataProvider metadataProvider, JobGenContext context, JobSpecification jobSpec,
             AbstractUnnestMapOperator unnestMap, IOperatorSchema opSchema, boolean retainInput, boolean retainMissing,
-            String datasetName, Dataset dataset, String indexName, ATypeTag searchKeyType, int[] keyFields,
-            SearchModifierType searchModifierType, IAlgebricksConstantValue similarityThreshold,
-            int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, boolean isFullTextSearchQuery, int frameLimit)
-            throws AlgebricksException {
-        boolean propagateIndexFilter = unnestMap.propagateIndexFilter();
+            IMissingWriterFactory nonMatchWriterFactory, String datasetName, Dataset dataset, String indexName,
+            ATypeTag searchKeyType, int[] keyFields, SearchModifierType searchModifierType,
+            IAlgebricksConstantValue similarityThreshold, boolean propagateIndexFilter,
+            IMissingWriterFactory nonFilterWriterFactory, int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes,
+            boolean isFullTextSearchQuery, int frameLimit) throws AlgebricksException {
         IAObject simThresh = ((AsterixConstantValue) similarityThreshold).getObject();
         int numPrimaryKeys = dataset.getPrimaryKeys().size();
         Index secondaryIndex = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(),
@@ -178,14 +186,14 @@
         IIndexDataflowHelperFactory dataflowHelperFactory = new IndexDataflowHelperFactory(
                 metadataProvider.getStorageComponentProvider().getStorageManager(), secondarySplitsAndConstraint.first);
 
-        LSMInvertedIndexSearchOperatorDescriptor invIndexSearchOp =
-                new LSMInvertedIndexSearchOperatorDescriptor(jobSpec, outputRecDesc, queryField, dataflowHelperFactory,
-                        queryTokenizerFactory, fullTextConfigEvaluatorFactory, searchModifierFactory, retainInput,
-                        retainMissing, context.getMissingWriterFactory(),
-                        dataset.getSearchCallbackFactory(metadataProvider.getStorageComponentProvider(), secondaryIndex,
-                                IndexOperation.SEARCH, null),
-                        minFilterFieldIndexes, maxFilterFieldIndexes, isFullTextSearchQuery, numPrimaryKeys,
-                        propagateIndexFilter, frameLimit);
+        LSMInvertedIndexSearchOperatorDescriptor invIndexSearchOp = new LSMInvertedIndexSearchOperatorDescriptor(
+                jobSpec, outputRecDesc, queryField, dataflowHelperFactory, queryTokenizerFactory,
+                fullTextConfigEvaluatorFactory, searchModifierFactory, retainInput, retainMissing,
+                nonMatchWriterFactory,
+                dataset.getSearchCallbackFactory(metadataProvider.getStorageComponentProvider(), secondaryIndex,
+                        IndexOperation.SEARCH, null),
+                minFilterFieldIndexes, maxFilterFieldIndexes, isFullTextSearchQuery, numPrimaryKeys,
+                propagateIndexFilter, nonFilterWriterFactory, frameLimit);
         return new Pair<>(invIndexSearchOp, secondarySplitsAndConstraint.second);
     }
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/RTreeSearchPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/RTreeSearchPOperator.java
index a80e5b3..6534ebe 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/RTreeSearchPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/RTreeSearchPOperator.java
@@ -42,10 +42,12 @@
 import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSourceIndex;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractUnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterUnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
 import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
+import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
 
 /**
  * Contributes the runtime operator for an unnest-map representing a RTree
@@ -82,6 +84,7 @@
         int[] keyIndexes = getKeyIndexes(jobGenParams.getKeyVarList(), inputSchemas);
 
         boolean propagateIndexFilter = unnestMap.propagateIndexFilter();
+        IMissingWriterFactory nonFilterWriterFactory = getNonFilterWriterFactory(propagateIndexFilter, context);
         int[] minFilterFieldIndexes = getKeyIndexes(unnestMap.getMinFilterVars(), inputSchemas);
         int[] maxFilterFieldIndexes = getKeyIndexes(unnestMap.getMaxFilterVars(), inputSchemas);
 
@@ -93,15 +96,20 @@
             outputVars = new ArrayList<LogicalVariable>();
             VariableUtilities.getLiveVariables(unnestMap, outputVars);
         }
-        boolean retainNull = false;
+        boolean retainMissing = false;
+        IMissingWriterFactory nonMatchWriterFactory = null;
         if (op.getOperatorTag() == LogicalOperatorTag.LEFT_OUTER_UNNEST_MAP) {
             // By nature, LEFT_OUTER_UNNEST_MAP should generate null values for non-matching tuples.
-            retainNull = true;
+            retainMissing = true;
+            nonMatchWriterFactory = getNonMatchWriterFactory(((LeftOuterUnnestMapOperator) unnestMap).getMissingValue(),
+                    context, unnestMap.getSourceLocation());
         }
-        Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> rtreeSearch = mp.buildRtreeRuntime(
-                builder.getJobSpec(), outputVars, opSchema, typeEnv, context, jobGenParams.getRetainInput(), retainNull,
-                dataset, jobGenParams.getIndexName(), keyIndexes, propagateIndexFilter, minFilterFieldIndexes,
-                maxFilterFieldIndexes, unnestMap.getGenerateCallBackProceedResultVar());
+
+        Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> rtreeSearch =
+                mp.buildRtreeRuntime(builder.getJobSpec(), outputVars, opSchema, typeEnv, context,
+                        jobGenParams.getRetainInput(), retainMissing, nonMatchWriterFactory, dataset,
+                        jobGenParams.getIndexName(), keyIndexes, propagateIndexFilter, nonFilterWriterFactory,
+                        minFilterFieldIndexes, maxFilterFieldIndexes, unnestMap.getGenerateCallBackProceedResultVar());
         IOperatorDescriptor opDesc = rtreeSearch.first;
         opDesc.setSourceLocation(unnestMap.getSourceLocation());
 
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
index 1641665..5493dd1 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
@@ -151,7 +151,6 @@
 import org.apache.hyracks.algebricks.rewriter.rules.subplan.EliminateSubplanWithInputCardinalityOneRule;
 import org.apache.hyracks.algebricks.rewriter.rules.subplan.NestedSubplanToJoinRule;
 import org.apache.hyracks.algebricks.rewriter.rules.subplan.PushSubplanIntoGroupByRule;
-import org.apache.hyracks.algebricks.rewriter.rules.subplan.SubplanOutOfGroupRule;
 
 public final class RuleCollections {
 
@@ -241,7 +240,6 @@
         condPushDownAndJoinInference.add(new PushSubplanWithAggregateDownThroughProductRule());
         condPushDownAndJoinInference
                 .add(new AsterixPushMapOperatorThroughUnionRule(LogicalOperatorTag.ASSIGN, LogicalOperatorTag.SELECT));
-        condPushDownAndJoinInference.add(new SubplanOutOfGroupRule());
 
         condPushDownAndJoinInference.add(new AsterixExtractFunctionsFromJoinConditionRule());
 
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AsterixIntroduceGroupByCombinerRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AsterixIntroduceGroupByCombinerRule.java
index 83e62b8..df2e1ff 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AsterixIntroduceGroupByCombinerRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AsterixIntroduceGroupByCombinerRule.java
@@ -58,7 +58,7 @@
             ScalarFunctionCallExpression nonSystemNullTest =
                     new ScalarFunctionCallExpression(finfoNot, new MutableObject<>(isSystemNullTest));
             nonSystemNullTest.setSourceLocation(sourceLoc);
-            selectNonSystemNull = new SelectOperator(new MutableObject<>(nonSystemNullTest), false, null);
+            selectNonSystemNull = new SelectOperator(new MutableObject<>(nonSystemNullTest));
             selectNonSystemNull.setSourceLocation(sourceLoc);
         } else {
             List<Mutable<ILogicalExpression>> isSystemNullTestList = new ArrayList<>();
@@ -76,7 +76,7 @@
             }
             IFunctionInfo finfoAnd = context.getMetadataProvider().lookupFunction(AlgebricksBuiltinFunctions.AND);
             selectNonSystemNull = new SelectOperator(
-                    new MutableObject<>(new ScalarFunctionCallExpression(finfoAnd, isSystemNullTestList)), false, null);
+                    new MutableObject<>(new ScalarFunctionCallExpression(finfoAnd, isSystemNullTestList)));
             selectNonSystemNull.setSourceLocation(sourceLoc);
         }
 
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
index f8e84ad..03d112d 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
@@ -33,6 +33,7 @@
 import org.apache.asterix.common.exceptions.WarningCollector;
 import org.apache.asterix.dataflow.data.common.ExpressionTypeComputer;
 import org.apache.asterix.dataflow.data.nontagged.MissingWriterFactory;
+import org.apache.asterix.dataflow.data.nontagged.NullWriterFactory;
 import org.apache.asterix.formats.nontagged.ADMPrinterFactoryProvider;
 import org.apache.asterix.formats.nontagged.BinaryBooleanInspector;
 import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
@@ -125,7 +126,8 @@
         }
 
         @Override
-        public Object getVarType(LogicalVariable var, List<LogicalVariable> nonNullVariables,
+        public Object getVarType(LogicalVariable var, List<LogicalVariable> nonMissableVariables,
+                List<List<LogicalVariable>> correlatedMissableVariableLists, List<LogicalVariable> nonNullableVariables,
                 List<List<LogicalVariable>> correlatedNullableVariableLists) {
             throw new IllegalStateException();
         }
@@ -149,7 +151,7 @@
                 BinaryHashFunctionFactoryProvider.INSTANCE, BinaryHashFunctionFamilyProvider.INSTANCE,
                 BinaryComparatorFactoryProvider.INSTANCE, TypeTraitProvider.INSTANCE, BinaryBooleanInspector.FACTORY,
                 BinaryIntegerInspector.FACTORY, ADMPrinterFactoryProvider.INSTANCE, MissingWriterFactory.INSTANCE,
-                UnnestingPositionWriterFactory.INSTANCE, null,
+                NullWriterFactory.INSTANCE, UnnestingPositionWriterFactory.INSTANCE, null,
                 new ExpressionRuntimeProvider(new QueryLogicalExpressionJobGen(metadataProvider.getFunctionManager())),
                 ExpressionTypeComputer.INSTANCE, null, null, null, null, GlobalConfig.DEFAULT_FRAME_SIZE, null,
                 NoOpWarningCollector.INSTANCE, 0, new PhysicalOptimizationConfig());
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
index 01f93a4..0969fd8 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
@@ -1132,7 +1132,7 @@
         }
 
         public void applyFilteringExpression(Mutable<ILogicalExpression> filterExpression) throws AlgebricksException {
-            SelectOperator selectOperator = new SelectOperator(filterExpression, false, null);
+            SelectOperator selectOperator = new SelectOperator(filterExpression);
             selectOperator.setSourceLocation(sourceLoc);
             this.currentTop = introduceNewOp(currentTop, selectOperator, true);
         }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PullSelectOutOfSpatialJoin.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PullSelectOutOfSpatialJoin.java
index e395549..507906b 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PullSelectOutOfSpatialJoin.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PullSelectOutOfSpatialJoin.java
@@ -106,7 +106,7 @@
         }
         // pull up
         ILogicalExpression pulledCond = makeCondition(otherPredicates, context, op);
-        SelectOperator select = new SelectOperator(new MutableObject<ILogicalExpression>(pulledCond), false, null);
+        SelectOperator select = new SelectOperator(new MutableObject<ILogicalExpression>(pulledCond));
         select.setSourceLocation(op.getSourceLocation());
         ILogicalExpression newJoinCond = makeCondition(spatialVarVarComps, context, op);
         join.getCondition().setValue(newJoinCond);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveLeftOuterUnnestForLeftOuterJoinRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveLeftOuterUnnestForLeftOuterJoinRule.java
index 775a1df..f9f9a4c 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveLeftOuterUnnestForLeftOuterJoinRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveLeftOuterUnnestForLeftOuterJoinRule.java
@@ -22,6 +22,7 @@
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Objects;
 import java.util.Set;
 
 import org.apache.asterix.om.functions.BuiltinFunctions;
@@ -38,7 +39,9 @@
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
 import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
@@ -46,6 +49,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterUnnestOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 
 /**
@@ -55,7 +59,7 @@
  * left-outer-unnest $x <- scan-collection($y)
  *   group-by($k){
  *     aggregate $y <- listify($z1)
- *       select not(is-missing($z2))
+ *       select not(is-missing/null($z2))
  *         NTS
  *   }
  *     left outer join ($a=$b)
@@ -89,7 +93,7 @@
 
         // Checks whether the left outer unnest and the group-by operator are qualified for rewriting.
         Triple<Boolean, ILogicalExpression, ILogicalExpression> checkGbyResult =
-                checkUnnestAndGby(outerUnnest, gbyOperator);
+                checkUnnestAndGby(outerUnnest, gbyOperator, lojOperator);
         // The argument for listify and not(is-missing(...)) check should be variables.
         if (!checkGbyResult.first || checkGbyResult.second == null || !isVariableReference(checkGbyResult.second)
                 || checkGbyResult.third == null || !isVariableReference(checkGbyResult.third)) {
@@ -127,7 +131,7 @@
 
     // Checks left outer unnest and gby.
     private Triple<Boolean, ILogicalExpression, ILogicalExpression> checkUnnestAndGby(
-            LeftOuterUnnestOperator outerUnnest, GroupByOperator gbyOperator) {
+            LeftOuterUnnestOperator outerUnnest, GroupByOperator gbyOperator, LeftOuterJoinOperator lojOperator) {
         // Checks left outer unnest.
         Pair<Boolean, LogicalVariable> checkUnnestResult = checkUnnest(outerUnnest);
         if (!checkUnnestResult.first) {
@@ -136,7 +140,8 @@
 
         // Checks group-by.
         LogicalVariable varToUnnest = checkUnnestResult.second;
-        Triple<Boolean, ILogicalExpression, ILogicalExpression> checkGbyResult = checkGroupBy(gbyOperator, varToUnnest);
+        Triple<Boolean, ILogicalExpression, ILogicalExpression> checkGbyResult =
+                checkGroupBy(gbyOperator, varToUnnest, lojOperator.getMissingValue());
         if (!checkGbyResult.first) {
             return new Triple<>(false, null, null);
         }
@@ -159,7 +164,7 @@
 
     // Checks the group-by operator on top of the left outer join operator.
     private Triple<Boolean, ILogicalExpression, ILogicalExpression> checkGroupBy(GroupByOperator gbyOperator,
-            LogicalVariable varToUnnest) {
+            LogicalVariable varToUnnest, IAlgebricksConstantValue leftOuterMissingValue) {
         Pair<Boolean, ILogicalOperator> checkNestedPlanResult = checkNestedPlan(gbyOperator);
         if (!checkNestedPlanResult.first) {
             return new Triple<>(false, null, null);
@@ -182,7 +187,7 @@
             return new Triple<>(false, null, null);
         }
         SelectOperator select = (SelectOperator) rootInputOp;
-        Pair<Boolean, ILogicalExpression> conditionArgPair = checkSelect(select);
+        Pair<Boolean, ILogicalExpression> conditionArgPair = checkSelect(select, leftOuterMissingValue);
         return new Triple<>(true, listifyArgPair.second, conditionArgPair.second);
     }
 
@@ -222,7 +227,8 @@
     }
 
     // Checks the expression for the nested select operator inside the group-by operator.
-    private Pair<Boolean, ILogicalExpression> checkSelect(SelectOperator select) {
+    private Pair<Boolean, ILogicalExpression> checkSelect(SelectOperator select,
+            IAlgebricksConstantValue leftOuterMissingValue) {
         ILogicalExpression condition = select.getCondition().getValue();
         if (condition.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
             return new Pair<>(false, null);
@@ -236,7 +242,9 @@
             return new Pair<>(false, null);
         }
         conditionFunc = (AbstractFunctionCallExpression) condition;
-        if (!conditionFunc.getFunctionIdentifier().equals(BuiltinFunctions.IS_MISSING)) {
+        FunctionIdentifier isMissingNullFuncId =
+                Objects.requireNonNull(OperatorPropertiesUtil.getIsMissingNullFunction(leftOuterMissingValue));
+        if (!conditionFunc.getFunctionIdentifier().equals(isMissingNullFuncId)) {
             return new Pair<>(false, null);
         }
         ILogicalExpression conditionArg = conditionFunc.getArguments().get(0).getValue();
@@ -265,7 +273,7 @@
         lhs.add(outerUnnest.getVariable());
         VariableReferenceExpression listifyVarRef = new VariableReferenceExpression(listifyVar);
         listifyVarRef.setSourceLocation(gbyOperator.getSourceLocation());
-        rhs.add(new MutableObject<ILogicalExpression>(listifyVarRef));
+        rhs.add(new MutableObject<>(listifyVarRef));
         List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> gbyList = gbyOperator.getGroupByList();
         for (Pair<LogicalVariable, Mutable<ILogicalExpression>> gbyPair : gbyList) {
             lhs.add(gbyPair.first);
@@ -273,7 +281,9 @@
         }
         AssignOperator assignOp = new AssignOperator(lhs, rhs);
         assignOp.setSourceLocation(outerUnnest.getSourceLocation());
-        assignOp.getInputs().add(new MutableObject<ILogicalOperator>(lojOperator));
+        assignOp.getInputs().add(new MutableObject<>(lojOperator));
+        lojOperator.setMissingValue(outerUnnest.getMissingValue());
+        context.computeAndSetTypeEnvironmentForOperator(lojOperator);
         context.computeAndSetTypeEnvironmentForOperator(assignOp);
         opRef.setValue(assignOp);
     }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
index 819922b..7f1ff4a 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
@@ -397,7 +397,10 @@
                                 }
 
                                 @Override
-                                public Object getVarType(LogicalVariable var, List<LogicalVariable> nonNullVariables,
+                                public Object getVarType(LogicalVariable var,
+                                        List<LogicalVariable> nonMissableVariables,
+                                        List<List<LogicalVariable>> correlatedMissableVariableLists,
+                                        List<LogicalVariable> nonNullableVariables,
                                         List<List<LogicalVariable>> correlatedNullableVariableLists)
                                         throws AlgebricksException {
                                     if (var.equals(optFuncExpr.getSourceVar(exprAndVarIdx.second))) {
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodAnalysisContext.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodAnalysisContext.java
index b066339..3d4986d 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodAnalysisContext.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodAnalysisContext.java
@@ -57,7 +57,7 @@
     // See AccessMethodUtils#removeUnjoinedDuplicatesInLOJ() for a definition of a special GroupBy
     // and extra output processing steps needed when it's not available.
     private Mutable<ILogicalOperator> lojSpecialGroupByOpRef = null;
-    private ScalarFunctionCallExpression lojIsMissingFuncInSpecialGroupBy = null;
+    private ScalarFunctionCallExpression lojIsMissingNullFuncInSpecialGroupBy = null;
 
     // For a secondary index, if we use only PK and secondary key field in a plan, it is an index-only plan.
     // Contains information about index-only plan
@@ -146,12 +146,12 @@
         return lojSpecialGroupByOpRef;
     }
 
-    public void setLOJIsMissingFuncInSpecialGroupBy(ScalarFunctionCallExpression isMissingFunc) {
-        lojIsMissingFuncInSpecialGroupBy = isMissingFunc;
+    public void setLOJIsMissingNullFuncInSpecialGroupBy(ScalarFunctionCallExpression isMissingNullFunc) {
+        lojIsMissingNullFuncInSpecialGroupBy = isMissingNullFunc;
     }
 
-    public ScalarFunctionCallExpression getLOJIsMissingFuncInSpecialGroupBy() {
-        return lojIsMissingFuncInSpecialGroupBy;
+    public ScalarFunctionCallExpression getLOJIsMissingNullFuncInSpecialGroupBy() {
+        return lojIsMissingNullFuncInSpecialGroupBy;
     }
 
     public Dataset getDatasetFromIndexDatasetMap(Index idx) {
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
index 063d55f..67e3784 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
@@ -83,6 +83,7 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
@@ -827,26 +828,28 @@
             Mutable<ILogicalOperator> joinRef, OptimizableOperatorSubTree indexSubTree,
             OptimizableOperatorSubTree probeSubTree, AccessMethodAnalysisContext analysisCtx,
             IOptimizationContext context, boolean isLeftOuterJoin, boolean isLeftOuterJoinWithSpecialGroupBy,
-            ILogicalOperator indexSearchOp, LogicalVariable newNullPlaceHolderVar,
-            Mutable<ILogicalExpression> conditionRef, Dataset dataset, Index chosenIndex) throws AlgebricksException {
+            IAlgebricksConstantValue leftOuterMissingValue, ILogicalOperator indexSearchOp,
+            LogicalVariable newMissingNullPlaceHolderVar, Mutable<ILogicalExpression> conditionRef, Dataset dataset,
+            Index chosenIndex) throws AlgebricksException {
         boolean isIndexOnlyPlan = analysisCtx.getIndexOnlyPlanInfo().getFirst();
         List<LogicalVariable> probePKVars = null;
         ILogicalOperator finalIndexSearchOp = indexSearchOp;
         if (isLeftOuterJoin) {
             if (isLeftOuterJoinWithSpecialGroupBy) {
-                ScalarFunctionCallExpression lojFuncExprs = analysisCtx.getLOJIsMissingFuncInSpecialGroupBy();
-                List<LogicalVariable> lojMissingVariables = new ArrayList<>();
-                lojFuncExprs.getUsedVariables(lojMissingVariables);
-                boolean lojMissingVarExist = !lojMissingVariables.isEmpty();
+                ScalarFunctionCallExpression lojFuncExprs = analysisCtx.getLOJIsMissingNullFuncInSpecialGroupBy();
+                List<LogicalVariable> lojMissingNullVariables = new ArrayList<>();
+                lojFuncExprs.getUsedVariables(lojMissingNullVariables);
+                boolean lojMissingNullVarExist = !lojMissingNullVariables.isEmpty();
 
                 // Resets the missing place holder variable.
-                AccessMethodUtils.resetLOJMissingPlaceholderVarInGroupByOp(analysisCtx, newNullPlaceHolderVar, context);
+                AccessMethodUtils.resetLOJMissingNullPlaceholderVarInGroupByOp(analysisCtx,
+                        newMissingNullPlaceHolderVar, context);
 
-                // For the index-only plan, if newNullPlaceHolderVar is not in the variable map of the union operator
+                // For the index-only plan, if newMissingNullPlaceHolderVar is not in the variable map of the union operator
                 // or if the variable is removed during the above method, we need to refresh the variable mapping in UNION.
                 if (isIndexOnlyPlan) {
                     finalIndexSearchOp = AccessMethodUtils.resetVariableMappingInUnionOpInIndexOnlyPlan(
-                            lojMissingVarExist, lojMissingVariables, indexSearchOp, afterJoinRefs, context);
+                            lojMissingNullVarExist, lojMissingNullVariables, indexSearchOp, afterJoinRefs, context);
                 }
             } else {
                 // We'll need to remove unjoined duplicates after the left outer join if there is no special GroupBy,
@@ -919,7 +922,9 @@
             } else {
                 // Non-index only plan case
                 indexSubTree.getDataSourceRef().setValue(finalIndexSearchOp);
-                SelectOperator topSelectOp = new SelectOperator(conditionRef, isLeftOuterJoin, newNullPlaceHolderVar);
+                SelectOperator topSelectOp = isLeftOuterJoin
+                        ? new SelectOperator(conditionRef, leftOuterMissingValue, newMissingNullPlaceHolderVar)
+                        : new SelectOperator(conditionRef);
                 topSelectOp.setSourceLocation(sourceLoc);
                 topSelectOp.getInputs().add(indexSubTree.getRootRef());
                 topSelectOp.setExecutionMode(ExecutionMode.LOCAL);
@@ -935,7 +940,8 @@
         }
 
         if (isLeftOuterJoin && !isLeftOuterJoinWithSpecialGroupBy) {
-            finalOp = removeUnjoinedDuplicatesInLOJ(finalOp, probePKVars, newNullPlaceHolderVar, context, sourceLoc);
+            finalOp = removeUnjoinedDuplicatesInLOJ(finalOp, probePKVars, newMissingNullPlaceHolderVar,
+                    leftOuterMissingValue, context, sourceLoc);
         }
 
         joinRef.setValue(finalOp);
@@ -947,12 +953,12 @@
      * (see {@link IntroduceJoinAccessMethodRule#checkAndApplyJoinTransformation(Mutable, IOptimizationContext)}.
      * A "Special GroupBy" is a GroupBy that eliminates unjoined duplicates that might be produced by the secondary
      * index probe. We probe secondary indexes on each index partition and return a tuple with a right branch variable
-     * set to MISSING if there's no match on that partition. Therefore if there's more than one partition where
-     * nothing joined then the index probe plan will produce several such MISSING tuples, however the join result
-     * must have a single MISSING tuple for each unjoined left branch tuple. If the special GroupBy is available then
-     * it'll eliminate those MISSING duplicates, otherwise this method is called to produce additional operators to
-     * achieve that. The special GroupBy operators are introduced by the optimizer when rewriting FROM-LET or
-     * equivalent patterns into a left outer join with parent a group by.
+     * set to MISSING (or NULL) if there's no match on that partition. Therefore if there's more than one partition
+     * where nothing joined then the index probe plan will produce several such MISSING (or NULL) tuples, however the
+     * join result must have a single MISSING (or NULL) tuple for each unjoined left branch tuple. If the special
+     * GroupBy is available then it'll eliminate those MISSING (or NULL) duplicates, otherwise this method is
+     * called to produce additional operators to achieve that. The special GroupBy operators are introduced by
+     * the optimizer when rewriting FROM-LET or equivalent patterns into a left outer join with parent a group by.
      * <p>
      * The plan generated by this method to eliminate unjoined duplicates is as follows:
      * <ul>
@@ -970,8 +976,9 @@
      * the Select operator eliminates those unjoined duplicate tuples.
      */
     private static SelectOperator removeUnjoinedDuplicatesInLOJ(ILogicalOperator inputOp,
-            List<LogicalVariable> probePKVars, LogicalVariable newNullPlaceHolderVar, IOptimizationContext context,
-            SourceLocation sourceLoc) throws AlgebricksException {
+            List<LogicalVariable> probePKVars, LogicalVariable newNullPlaceHolderVar,
+            IAlgebricksConstantValue lojMissingValue, IOptimizationContext context, SourceLocation sourceLoc)
+            throws AlgebricksException {
         if (probePKVars == null || probePKVars.isEmpty()) {
             throw new IllegalArgumentException();
         }
@@ -984,16 +991,17 @@
 
         VariableReferenceExpression winOrderByVarRef = new VariableReferenceExpression(newNullPlaceHolderVar);
         winOrderByVarRef.setSourceLocation(sourceLoc);
-        /* Sort in DESC order, so all MISSING values are at the end */
+        /* Sort in DESC order, so all MISSING (or NULL) values are at the end */
         Pair<OrderOperator.IOrder, Mutable<ILogicalExpression>> winOrderByPair =
                 new Pair<>(OrderOperator.DESC_ORDER, new MutableObject<>(winOrderByVarRef));
 
         LogicalVariable winVar = context.newVar();
         VariableReferenceExpression winOrderByVarRef2 = new VariableReferenceExpression(newNullPlaceHolderVar);
         winOrderByVarRef2.setSourceLocation(sourceLoc);
-        AbstractFunctionCallExpression winExpr =
-                BuiltinFunctions.makeWindowFunctionExpression(BuiltinFunctions.WIN_MARK_FIRST_MISSING_IMPL,
-                        Collections.singletonList(new MutableObject<>(winOrderByVarRef2)));
+        FunctionIdentifier winMarkFirstUnknownValueFn = lojMissingValue.isNull()
+                ? BuiltinFunctions.WIN_MARK_FIRST_NULL_IMPL : BuiltinFunctions.WIN_MARK_FIRST_MISSING_IMPL;
+        AbstractFunctionCallExpression winExpr = BuiltinFunctions.makeWindowFunctionExpression(
+                winMarkFirstUnknownValueFn, Collections.singletonList(new MutableObject<>(winOrderByVarRef2)));
 
         WindowOperator winOp = new WindowOperator(winPartitionByList, Collections.singletonList(winOrderByPair));
         winOp.getVariables().add(winVar);
@@ -1005,7 +1013,7 @@
 
         VariableReferenceExpression winVarRef = new VariableReferenceExpression(winVar);
         winVarRef.setSourceLocation(sourceLoc);
-        SelectOperator selectOp = new SelectOperator(new MutableObject<>(winVarRef), false, null);
+        SelectOperator selectOp = new SelectOperator(new MutableObject<>(winVarRef));
         selectOp.getInputs().add(new MutableObject<>(winOp));
         selectOp.setExecutionMode(ExecutionMode.LOCAL);
         selectOp.setSourceLocation(sourceLoc);
@@ -1017,7 +1025,8 @@
     public static ILogicalOperator createSecondaryIndexUnnestMap(Dataset dataset, ARecordType recordType,
             ARecordType metaRecordType, Index index, ILogicalOperator inputOp, AccessMethodJobGenParams jobGenParams,
             IOptimizationContext context, boolean retainInput, boolean retainNull,
-            boolean generateInstantTrylockResultFromIndexSearch) throws AlgebricksException {
+            boolean generateInstantTrylockResultFromIndexSearch, IAlgebricksConstantValue leftOuterMissingValue)
+            throws AlgebricksException {
         SourceLocation sourceLoc = inputOp.getSourceLocation();
         // The job gen parameters are transferred to the actual job gen via the UnnestMapOperator's function arguments.
         ArrayList<Mutable<ILogicalExpression>> secondaryIndexFuncArgs = new ArrayList<>();
@@ -1047,7 +1056,7 @@
             if (retainInput) {
                 LeftOuterUnnestMapOperator secondaryIndexLeftOuterUnnestOp = new LeftOuterUnnestMapOperator(
                         secondaryIndexUnnestVars, new MutableObject<ILogicalExpression>(secondaryIndexSearchFunc),
-                        secondaryIndexOutputTypes, true);
+                        secondaryIndexOutputTypes, leftOuterMissingValue);
                 secondaryIndexLeftOuterUnnestOp.setSourceLocation(sourceLoc);
                 secondaryIndexLeftOuterUnnestOp
                         .setGenerateCallBackProceedResultVar(generateInstantTrylockResultFromIndexSearch);
@@ -1078,7 +1087,8 @@
             ILogicalOperator inputOp, IOptimizationContext context, boolean sortPrimaryKeys, boolean retainInput,
             boolean retainMissing, boolean requiresBroadcast, boolean requiresDistinct,
             List<LogicalVariable> primaryKeyVars, List<LogicalVariable> primaryIndexUnnestVars,
-            List<LogicalVariable> auxDistinctVars, List<Object> primaryIndexOutputTypes) throws AlgebricksException {
+            List<LogicalVariable> auxDistinctVars, List<Object> primaryIndexOutputTypes,
+            IAlgebricksConstantValue leftOuterMissingValue) throws AlgebricksException {
         SourceLocation sourceLoc = inputOp.getSourceLocation();
 
         // Sanity check: requiresDistinct and sortPrimaryKeys are mutually exclusive.
@@ -1130,11 +1140,11 @@
         // Creates the primary-index search unnest-map operator.
         AbstractUnnestMapOperator primaryIndexUnnestMapOp =
                 createPrimaryIndexUnnestMapOp(dataset, retainInput, retainMissing, requiresBroadcast, primaryKeyVars,
-                        primaryIndexUnnestVars, primaryIndexOutputTypes, sourceLoc);
+                        primaryIndexUnnestVars, primaryIndexOutputTypes, sourceLoc, leftOuterMissingValue);
         if (requiresDistinct) {
-            primaryIndexUnnestMapOp.getInputs().add(new MutableObject<ILogicalOperator>(distinct));
+            primaryIndexUnnestMapOp.getInputs().add(new MutableObject<>(distinct));
         } else if (sortPrimaryKeys) {
-            primaryIndexUnnestMapOp.getInputs().add(new MutableObject<ILogicalOperator>(order));
+            primaryIndexUnnestMapOp.getInputs().add(new MutableObject<>(order));
         } else {
             primaryIndexUnnestMapOp.getInputs().add(new MutableObject<>(inputOp));
         }
@@ -1149,9 +1159,9 @@
             ARecordType metaRecordType, ILogicalOperator inputOp, IOptimizationContext context, boolean retainInput,
             boolean retainMissing, boolean requiresBroadcast, Index secondaryIndex,
             AccessMethodAnalysisContext analysisCtx, OptimizableOperatorSubTree subTree,
-            LogicalVariable newMissingPlaceHolderForLOJ, List<LogicalVariable> pkVarsFromSIdxUnnestMapOp,
-            List<LogicalVariable> primaryIndexUnnestVars, List<Object> primaryIndexOutputTypes)
-            throws AlgebricksException {
+            LogicalVariable newMissingPlaceHolderForLOJ, IAlgebricksConstantValue leftOuterMissingValue,
+            List<LogicalVariable> pkVarsFromSIdxUnnestMapOp, List<LogicalVariable> primaryIndexUnnestVars,
+            List<Object> primaryIndexOutputTypes) throws AlgebricksException {
         SourceLocation sourceLoc = inputOp.getSourceLocation();
         Quadruple<Boolean, Boolean, Boolean, Boolean> indexOnlyPlanInfo = analysisCtx.getIndexOnlyPlanInfo();
         // From now on, we deal with the index-only plan.
@@ -1527,7 +1537,7 @@
         // The job gen parameters are transferred to the actual job gen via the UnnestMapOperator's function arguments.
         AbstractUnnestMapOperator primaryIndexUnnestMapOp = createPrimaryIndexUnnestMapOp(dataset, retainInput,
                 retainMissing, requiresBroadcast, pkVarsInLeftPathFromSIdxSearchBeforeSplit,
-                pkVarsFromPIdxSearchInLeftPath, primaryIndexOutputTypes, sourceLoc);
+                pkVarsFromPIdxSearchInLeftPath, primaryIndexOutputTypes, sourceLoc, leftOuterMissingValue);
         primaryIndexUnnestMapOp.setSourceLocation(sourceLoc);
         primaryIndexUnnestMapOp.getInputs().add(new MutableObject<ILogicalOperator>(origVarsToLeftPathVarsAssignOp));
         context.computeAndSetTypeEnvironmentForOperator(primaryIndexUnnestMapOp);
@@ -1544,8 +1554,9 @@
         ILogicalExpression conditionRefExpr = conditionRef.getValue().cloneExpression();
         // The retainMissing variable contains the information whether we are optimizing a left-outer join or not.
         LogicalVariable newMissingPlaceHolderVar = retainMissing ? newMissingPlaceHolderForLOJ : null;
-        newSelectOpInLeftPath = new SelectOperator(new MutableObject<ILogicalExpression>(conditionRefExpr),
-                retainMissing, newMissingPlaceHolderVar);
+        newSelectOpInLeftPath =
+                retainMissing ? new SelectOperator(new MutableObject<>(conditionRefExpr), leftOuterMissingValue,
+                        newMissingPlaceHolderVar) : new SelectOperator(new MutableObject<>(conditionRefExpr));
         newSelectOpInLeftPath.setSourceLocation(conditionRefExpr.getSourceLocation());
         VariableUtilities.substituteVariables(newSelectOpInLeftPath, origVarToNewVarInLeftPathMap, context);
 
@@ -1604,8 +1615,11 @@
             // since we need to change the variable reference in the SELECT operator.
             // For the index-nested-loop join case, we copy the condition of the join operator.
             ILogicalExpression conditionRefExpr2 = conditionRef.getValue().cloneExpression();
-            newSelectOpInRightPath = new SelectOperator(new MutableObject<ILogicalExpression>(conditionRefExpr2),
-                    retainMissing, newMissingPlaceHolderVar);
+            newSelectOpInRightPath =
+                    retainMissing
+                            ? new SelectOperator(new MutableObject<>(conditionRefExpr2), leftOuterMissingValue,
+                                    newMissingPlaceHolderVar)
+                            : new SelectOperator(new MutableObject<>(conditionRefExpr2));
             newSelectOpInRightPath.setSourceLocation(conditionRefExpr2.getSourceLocation());
             newSelectOpInRightPath.getInputs().add(new MutableObject<ILogicalOperator>(currentTopOpInRightPath));
             VariableUtilities.substituteVariables(newSelectOpInRightPath, origVarToSIdxUnnestMapOpVarMap, context);
@@ -1655,7 +1669,7 @@
     private static AbstractUnnestMapOperator createPrimaryIndexUnnestMapOp(Dataset dataset, boolean retainInput,
             boolean retainMissing, boolean requiresBroadcast, List<LogicalVariable> primaryKeyVars,
             List<LogicalVariable> primaryIndexUnnestVars, List<Object> primaryIndexOutputTypes,
-            SourceLocation sourceLoc) throws AlgebricksException {
+            SourceLocation sourceLoc, IAlgebricksConstantValue leftOuterMissingValue) throws AlgebricksException {
         // The job gen parameters are transferred to the actual job gen via the UnnestMapOperator's function arguments.
         List<Mutable<ILogicalExpression>> primaryIndexFuncArgs = new ArrayList<>();
         BTreeJobGenParams jobGenParams = new BTreeJobGenParams(dataset.getDatasetName(), IndexType.BTREE,
@@ -1675,12 +1689,11 @@
         // This is the operator that jobgen will be looking for. It contains an unnest function that has
         // all necessary arguments to determine which index to use, which variables contain the index-search keys,
         // what is the original dataset, etc.
-        AbstractUnnestMapOperator primaryIndexUnnestMapOp = null;
+        AbstractUnnestMapOperator primaryIndexUnnestMapOp;
         if (retainMissing) {
             if (retainInput) {
                 primaryIndexUnnestMapOp = new LeftOuterUnnestMapOperator(primaryIndexUnnestVars,
-                        new MutableObject<ILogicalExpression>(primaryIndexSearchFunc), primaryIndexOutputTypes,
-                        retainInput);
+                        new MutableObject<>(primaryIndexSearchFunc), primaryIndexOutputTypes, leftOuterMissingValue);
                 primaryIndexUnnestMapOp.setSourceLocation(sourceLoc);
             } else {
                 // Left-outer-join without retainNull and retainInput doesn't make sense.
@@ -1689,8 +1702,7 @@
             }
         } else {
             primaryIndexUnnestMapOp = new UnnestMapOperator(primaryIndexUnnestVars,
-                    new MutableObject<ILogicalExpression>(primaryIndexSearchFunc), primaryIndexOutputTypes,
-                    retainInput);
+                    new MutableObject<>(primaryIndexSearchFunc), primaryIndexOutputTypes, retainInput);
             primaryIndexUnnestMapOp.setSourceLocation(sourceLoc);
         }
         return primaryIndexUnnestMapOp;
@@ -1721,7 +1733,8 @@
             IOptimizationContext context, boolean sortPrimaryKeys, boolean retainInput, boolean retainMissing,
             boolean requiresBroadcast, Index secondaryIndex, AccessMethodAnalysisContext analysisCtx,
             OptimizableOperatorSubTree indexSubTree, OptimizableOperatorSubTree probeSubTree,
-            LogicalVariable newMissingPlaceHolderForLOJ) throws AlgebricksException {
+            LogicalVariable newMissingPlaceHolderForLOJ, IAlgebricksConstantValue leftOuterMissingValue)
+            throws AlgebricksException {
         // Common part for the non-index-only plan and index-only plan
         // Variables and types for the primary-index search.
         List<LogicalVariable> primaryIndexUnnestVars = new ArrayList<>();
@@ -1767,14 +1780,14 @@
 
             return createFinalNonIndexOnlySearchPlan(dataset, inputOp, context, !isArrayIndex && sortPrimaryKeys,
                     retainInput, retainMissing, requiresBroadcast, isArrayIndex, pkVarsFromSIdxUnnestMapOp,
-                    primaryIndexUnnestVars, joinPKVars, primaryIndexOutputTypes);
+                    primaryIndexUnnestVars, joinPKVars, primaryIndexOutputTypes, leftOuterMissingValue);
         } else if (!isArrayIndex) {
             // Index-only plan case: creates a UNIONALL operator that has two paths after the secondary unnest-map op,
             // and returns it.
             return createFinalIndexOnlySearchPlan(afterTopOpRefs, topOpRef, conditionRef, assignsBeforeTopOpRef,
                     dataset, recordType, metaRecordType, inputOp, context, retainInput, retainMissing,
                     requiresBroadcast, secondaryIndex, analysisCtx, indexSubTree, newMissingPlaceHolderForLOJ,
-                    pkVarsFromSIdxUnnestMapOp, primaryIndexUnnestVars, primaryIndexOutputTypes);
+                    leftOuterMissingValue, pkVarsFromSIdxUnnestMapOp, primaryIndexUnnestVars, primaryIndexOutputTypes);
         } else {
             throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, inputOp.getSourceLocation(),
                     "Cannot use index-only plan with array indexes.");
@@ -1809,23 +1822,23 @@
         return createRectangleExpr;
     }
 
-    private static ScalarFunctionCallExpression getNestedIsMissingCall(AbstractFunctionCallExpression call,
-            OptimizableOperatorSubTree rightSubTree) throws AlgebricksException {
-        ScalarFunctionCallExpression isMissingFuncExpr;
+    private static ScalarFunctionCallExpression getNestedIsMissingNullCall(AbstractFunctionCallExpression call,
+            OptimizableOperatorSubTree rightSubTree, FunctionIdentifier funId) throws AlgebricksException {
+        ScalarFunctionCallExpression isMissingNullFuncExpr;
         if (call.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.NOT)) {
             if (call.getArguments().get(0).getValue().getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
                 if (((AbstractFunctionCallExpression) call.getArguments().get(0).getValue()).getFunctionIdentifier()
-                        .equals(AlgebricksBuiltinFunctions.IS_MISSING)) {
-                    isMissingFuncExpr = (ScalarFunctionCallExpression) call.getArguments().get(0).getValue();
-                    if (isMissingFuncExpr.getArguments().get(0).getValue()
+                        .equals(funId)) {
+                    isMissingNullFuncExpr = (ScalarFunctionCallExpression) call.getArguments().get(0).getValue();
+                    if (isMissingNullFuncExpr.getArguments().get(0).getValue()
                             .getExpressionTag() == LogicalExpressionTag.VARIABLE) {
                         LogicalVariable var =
-                                ((VariableReferenceExpression) isMissingFuncExpr.getArguments().get(0).getValue())
+                                ((VariableReferenceExpression) isMissingNullFuncExpr.getArguments().get(0).getValue())
                                         .getVariableReference();
                         List<LogicalVariable> liveSubplanVars = new ArrayList<>();
                         VariableUtilities.getSubplanLocalLiveVariables(rightSubTree.getRoot(), liveSubplanVars);
                         if (liveSubplanVars.contains(var)) {
-                            return isMissingFuncExpr;
+                            return isMissingNullFuncExpr;
                         }
                     }
                 }
@@ -1834,9 +1847,9 @@
         return null;
     }
 
-    public static ScalarFunctionCallExpression findIsMissingInSubplan(AbstractLogicalOperator inputOp,
-            OptimizableOperatorSubTree rightSubTree) throws AlgebricksException {
-        ScalarFunctionCallExpression isMissingFuncExpr = null;
+    public static ScalarFunctionCallExpression findIsMissingNullInSubplan(AbstractLogicalOperator inputOp,
+            OptimizableOperatorSubTree rightSubTree, FunctionIdentifier funId) throws AlgebricksException {
+        ScalarFunctionCallExpression isMissingNullFuncExpr = null;
         AbstractLogicalOperator currentOp = inputOp;
         while (currentOp != null) {
             if (currentOp.getOperatorTag() == LogicalOperatorTag.SELECT) {
@@ -1847,27 +1860,27 @@
                     if (call.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.AND)) {
                         for (Mutable<ILogicalExpression> mexpr : call.getArguments()) {
                             if (mexpr.getValue().getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
-                                isMissingFuncExpr = getNestedIsMissingCall(
-                                        (AbstractFunctionCallExpression) mexpr.getValue(), rightSubTree);
-                                if (isMissingFuncExpr != null) {
-                                    return isMissingFuncExpr;
+                                isMissingNullFuncExpr = getNestedIsMissingNullCall(
+                                        (AbstractFunctionCallExpression) mexpr.getValue(), rightSubTree, funId);
+                                if (isMissingNullFuncExpr != null) {
+                                    return isMissingNullFuncExpr;
                                 }
                             }
                         }
                     }
-                    isMissingFuncExpr = getNestedIsMissingCall(call, rightSubTree);
-                    if (isMissingFuncExpr != null) {
-                        return isMissingFuncExpr;
+                    isMissingNullFuncExpr = getNestedIsMissingNullCall(call, rightSubTree, funId);
+                    if (isMissingNullFuncExpr != null) {
+                        return isMissingNullFuncExpr;
                     }
                 }
             } else if (currentOp.hasNestedPlans()) {
                 AbstractOperatorWithNestedPlans nestedPlanOp = (AbstractOperatorWithNestedPlans) currentOp;
                 for (ILogicalPlan nestedPlan : nestedPlanOp.getNestedPlans()) {
                     for (Mutable<ILogicalOperator> root : nestedPlan.getRoots()) {
-                        isMissingFuncExpr =
-                                findIsMissingInSubplan((AbstractLogicalOperator) root.getValue(), rightSubTree);
-                        if (isMissingFuncExpr != null) {
-                            return isMissingFuncExpr;
+                        isMissingNullFuncExpr = findIsMissingNullInSubplan((AbstractLogicalOperator) root.getValue(),
+                                rightSubTree, funId);
+                        if (isMissingNullFuncExpr != null) {
+                            return isMissingNullFuncExpr;
                         }
                     }
                 }
@@ -1875,27 +1888,29 @@
             currentOp = currentOp.getInputs().isEmpty() ? null
                     : (AbstractLogicalOperator) currentOp.getInputs().get(0).getValue();
         }
-        return isMissingFuncExpr;
+        return isMissingNullFuncExpr;
     }
 
-    public static ScalarFunctionCallExpression findLOJIsMissingFuncInGroupBy(GroupByOperator lojGroupbyOp,
-            OptimizableOperatorSubTree rightSubTree) throws AlgebricksException {
-        //find IS_MISSING function of which argument has the nullPlaceholder variable in the nested plan of groupby.
+    public static ScalarFunctionCallExpression findLOJIsMissingNullFuncInGroupBy(GroupByOperator lojGroupbyOp,
+            OptimizableOperatorSubTree rightSubTree, FunctionIdentifier funId) throws AlgebricksException {
+        //find IS_MISSING or IS_NULL function of which argument has the nullPlaceholder variable in the nested plan of groupby.
         ALogicalPlanImpl subPlan = (ALogicalPlanImpl) lojGroupbyOp.getNestedPlans().get(0);
         Mutable<ILogicalOperator> subPlanRootOpRef = subPlan.getRoots().get(0);
         AbstractLogicalOperator subPlanRootOp = (AbstractLogicalOperator) subPlanRootOpRef.getValue();
-        return findIsMissingInSubplan(subPlanRootOp, rightSubTree);
+        return findIsMissingNullInSubplan(subPlanRootOp, rightSubTree, funId);
     }
 
-    public static void resetLOJMissingPlaceholderVarInGroupByOp(AccessMethodAnalysisContext analysisCtx,
-            LogicalVariable newMissingPlaceholderVaraible, IOptimizationContext context) throws AlgebricksException {
+    public static void resetLOJMissingNullPlaceholderVarInGroupByOp(AccessMethodAnalysisContext analysisCtx,
+            LogicalVariable newMissingNullPlaceholderVaraible, IOptimizationContext context)
+            throws AlgebricksException {
 
         //reset the missing placeholder variable in groupby operator
-        ScalarFunctionCallExpression isMissingFuncExpr = analysisCtx.getLOJIsMissingFuncInSpecialGroupBy();
-        isMissingFuncExpr.getArguments().clear();
-        VariableReferenceExpression newMissingVarRef = new VariableReferenceExpression(newMissingPlaceholderVaraible);
-        newMissingVarRef.setSourceLocation(isMissingFuncExpr.getSourceLocation());
-        isMissingFuncExpr.getArguments().add(new MutableObject<ILogicalExpression>(newMissingVarRef));
+        ScalarFunctionCallExpression isMissingNullFuncExpr = analysisCtx.getLOJIsMissingNullFuncInSpecialGroupBy();
+        isMissingNullFuncExpr.getArguments().clear();
+        VariableReferenceExpression newMissingNullVarRef =
+                new VariableReferenceExpression(newMissingNullPlaceholderVaraible);
+        newMissingNullVarRef.setSourceLocation(isMissingNullFuncExpr.getSourceLocation());
+        isMissingNullFuncExpr.getArguments().add(new MutableObject<ILogicalExpression>(newMissingNullVarRef));
 
         //recompute type environment.
         OperatorPropertiesUtil.typeOpRec(analysisCtx.getLOJSpecialGroupByOpRef(), context);
@@ -2754,10 +2769,10 @@
      *
      * @throws AlgebricksException
      */
-    public static ILogicalOperator resetVariableMappingInUnionOpInIndexOnlyPlan(boolean LOJVarExist,
-            List<LogicalVariable> LOJMissingVariables, ILogicalOperator unionAllOp,
+    public static ILogicalOperator resetVariableMappingInUnionOpInIndexOnlyPlan(boolean lojVarExist,
+            List<LogicalVariable> lojMissingNullVariables, ILogicalOperator unionAllOp,
             List<Mutable<ILogicalOperator>> aboveTopRefs, IOptimizationContext context) throws AlgebricksException {
-        // For an index-only plan, if newNullPlaceHolderVar is not in the variable map of the UNIONALL operator,
+        // For an index-only plan, if newMissingNullPlaceHolderVar is not in the variable map of the UNIONALL operator,
         // we need to add this variable to the map.
         // Also, we need to delete replaced variables in the map if it was used only in the group-by operator.
         if (unionAllOp.getOperatorTag() != LogicalOperatorTag.UNIONALL) {
@@ -2766,15 +2781,15 @@
 
         // First, check whether the given old variable can be deleted. If it is used somewhere else
         // except the group-by operator, we can't delete it since we need to propagate it.
-        boolean LOJVarCanBeDeleted = true;
-        if (LOJVarExist) {
+        boolean lojVarCanBeDeleted = true;
+        if (lojVarExist) {
             List<LogicalVariable> usedVars = new ArrayList<>();
             for (int i = 0; i < aboveTopRefs.size(); i++) {
                 usedVars.clear();
                 ILogicalOperator lOp = aboveTopRefs.get(i).getValue();
                 VariableUtilities.getUsedVariables(lOp, usedVars);
-                if (usedVars.containsAll(LOJMissingVariables) && lOp.getOperatorTag() != LogicalOperatorTag.GROUP) {
-                    LOJVarCanBeDeleted = false;
+                if (usedVars.containsAll(lojMissingNullVariables) && lOp.getOperatorTag() != LogicalOperatorTag.GROUP) {
+                    lojVarCanBeDeleted = false;
                     break;
                 }
             }
@@ -2783,20 +2798,20 @@
         List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> varMap =
                 ((UnionAllOperator) unionAllOp).getVariableMappings();
 
-        if (LOJVarExist && LOJVarCanBeDeleted) {
+        if (lojVarExist && lojVarCanBeDeleted) {
             // Delete old variables from the map.
             for (Iterator<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> it = varMap.iterator(); it
                     .hasNext();) {
                 Triple<LogicalVariable, LogicalVariable, LogicalVariable> tripleVars = it.next();
-                if (tripleVars.first.equals(LOJMissingVariables.get(0))
-                        || tripleVars.second.equals(LOJMissingVariables.get(0))
-                        || tripleVars.third.equals(LOJMissingVariables.get(0))) {
+                if (tripleVars.first.equals(lojMissingNullVariables.get(0))
+                        || tripleVars.second.equals(lojMissingNullVariables.get(0))
+                        || tripleVars.third.equals(lojMissingNullVariables.get(0))) {
                     it.remove();
                 }
             }
         }
 
-        if (LOJVarExist && LOJVarCanBeDeleted) {
+        if (lojVarExist && lojVarCanBeDeleted) {
             UnionAllOperator newUnionAllOp = new UnionAllOperator(varMap);
             newUnionAllOp.getInputs()
                     .add(new MutableObject<ILogicalOperator>(unionAllOp.getInputs().get(0).getValue()));
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/ArrayBTreeAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/ArrayBTreeAccessMethod.java
index c633559..ce25d4d 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/ArrayBTreeAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/ArrayBTreeAccessMethod.java
@@ -43,6 +43,8 @@
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterUnnestOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
@@ -85,8 +87,8 @@
     public boolean applyJoinPlanTransformation(List<Mutable<ILogicalOperator>> afterJoinRefs,
             Mutable<ILogicalOperator> joinRef, OptimizableOperatorSubTree leftSubTree,
             OptimizableOperatorSubTree rightSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
-            IOptimizationContext context, boolean isLeftOuterJoin, boolean isLeftOuterJoinWithSpecialGroupBy)
-            throws AlgebricksException {
+            IOptimizationContext context, boolean isLeftOuterJoin, boolean isLeftOuterJoinWithSpecialGroupBy,
+            IAlgebricksConstantValue leftOuterMissingValue) throws AlgebricksException {
         AbstractBinaryJoinOperator joinOp = (AbstractBinaryJoinOperator) joinRef.getValue();
         Mutable<ILogicalExpression> conditionRef = joinOp.getCondition();
         Dataset dataset = analysisCtx.getIndexDatasetMap().get(chosenIndex);
@@ -159,7 +161,8 @@
                 if (workingOp.getOperatorTag() == LogicalOperatorTag.UNNEST) {
                     UnnestOperator oldUnnest = (UnnestOperator) workingOp;
                     LeftOuterUnnestOperator newUnnest = new LeftOuterUnnestOperator(oldUnnest.getVariable(),
-                            new MutableObject<>(oldUnnest.getExpressionRef().getValue()));
+                            new MutableObject<>(oldUnnest.getExpressionRef().getValue()),
+                            ConstantExpression.MISSING.getValue());
                     newUnnest.setSourceLocation(oldUnnest.getSourceLocation());
                     newUnnest.getInputs().addAll(oldUnnest.getInputs());
                     newUnnest.setExecutionMode(oldUnnest.getExecutionMode());
@@ -185,14 +188,14 @@
 
         ILogicalOperator indexSearchOp = createIndexSearchPlan(afterJoinRefs, joinRef, conditionRef,
                 indexSubTree.getAssignsAndUnnestsRefs(), indexSubTree, probeSubTree, chosenIndex, analysisCtx, true,
-                isLeftOuterJoin, true, context, newNullPlaceHolderVar);
+                isLeftOuterJoin, true, context, newNullPlaceHolderVar, leftOuterMissingValue);
         if (indexSearchOp == null) {
             return false;
         }
 
         return AccessMethodUtils.finalizeJoinPlanTransformation(afterJoinRefs, joinRef, indexSubTree, probeSubTree,
-                analysisCtx, context, isLeftOuterJoin, isLeftOuterJoinWithSpecialGroupBy, indexSearchOp,
-                newNullPlaceHolderVar, conditionRef, dataset, chosenIndex);
+                analysisCtx, context, isLeftOuterJoin, isLeftOuterJoinWithSpecialGroupBy, leftOuterMissingValue,
+                indexSearchOp, newNullPlaceHolderVar, conditionRef, dataset, chosenIndex);
     }
 
     @Override
@@ -201,7 +204,8 @@
             List<Mutable<ILogicalOperator>> assignBeforeTheOpRefs, OptimizableOperatorSubTree indexSubTree,
             OptimizableOperatorSubTree probeSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
             boolean retainInput, boolean retainMissing, boolean requiresBroadcast, IOptimizationContext context,
-            LogicalVariable newMissingPlaceHolderForLOJ) throws AlgebricksException {
+            LogicalVariable newMissingNullPlaceHolderForLOJ, IAlgebricksConstantValue leftOuterMissingValue)
+            throws AlgebricksException {
 
         Index.ArrayIndexDetails chosenIndexDetails = (Index.ArrayIndexDetails) chosenIndex.getIndexDetails();
         List<List<String>> chosenIndexKeyFieldNames = new ArrayList<>();
@@ -218,8 +222,8 @@
 
         return createBTreeIndexSearchPlan(afterTopOpRefs, topOpRef, conditionRef, assignBeforeTheOpRefs, indexSubTree,
                 probeSubTree, chosenIndex, analysisCtx, retainInput, retainMissing, requiresBroadcast, context,
-                newMissingPlaceHolderForLOJ, chosenIndexKeyFieldNames, chosenIndexKeyFieldTypes,
-                chosenIndexKeyFieldSourceIndicators);
+                newMissingNullPlaceHolderForLOJ, leftOuterMissingValue, chosenIndexKeyFieldNames,
+                chosenIndexKeyFieldTypes, chosenIndexKeyFieldSourceIndicators);
     }
 
     @Override
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
index 1acbd89..60dd53b 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
@@ -64,6 +64,7 @@
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
@@ -199,7 +200,7 @@
                         afterSelectRefs),
                 false, subTree.getDataSourceRef().getValue().getInputs().get(0).getValue()
                         .getExecutionMode() == ExecutionMode.UNPARTITIONED,
-                context, null);
+                context, null, null);
 
         if (primaryIndexUnnestOp == null) {
             return false;
@@ -252,8 +253,8 @@
     public boolean applyJoinPlanTransformation(List<Mutable<ILogicalOperator>> afterJoinRefs,
             Mutable<ILogicalOperator> joinRef, OptimizableOperatorSubTree leftSubTree,
             OptimizableOperatorSubTree rightSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
-            IOptimizationContext context, boolean isLeftOuterJoin, boolean isLeftOuterJoinWithSpecialGroupBy)
-            throws AlgebricksException {
+            IOptimizationContext context, boolean isLeftOuterJoin, boolean isLeftOuterJoinWithSpecialGroupBy,
+            IAlgebricksConstantValue leftOuterMissingValue) throws AlgebricksException {
         AbstractBinaryJoinOperator joinOp = (AbstractBinaryJoinOperator) joinRef.getValue();
         Mutable<ILogicalExpression> conditionRef = joinOp.getCondition();
 
@@ -278,12 +279,12 @@
             return false;
         }
 
-        LogicalVariable newNullPlaceHolderVar = null;
+        LogicalVariable newMissingNullPlaceHolderVar = null;
         if (isLeftOuterJoin) {
-            // Gets a new null place holder variable that is the first field variable of the primary key
+            // Gets a new missing/null place holder variable that is the first field variable of the primary key
             // from the indexSubTree's datasourceScanOp.
             // We need this for all left outer joins, even those that do not have a special GroupBy
-            newNullPlaceHolderVar = indexSubTree.getDataSourceVariables().get(0);
+            newMissingNullPlaceHolderVar = indexSubTree.getDataSourceVariables().get(0);
         }
 
         boolean canContinue = AccessMethodUtils.setIndexOnlyPlanInfo(afterJoinRefs, joinRef, probeSubTree, indexSubTree,
@@ -294,15 +295,15 @@
 
         ILogicalOperator indexSearchOp = createIndexSearchPlan(afterJoinRefs, joinRef, conditionRef,
                 indexSubTree.getAssignsAndUnnestsRefs(), indexSubTree, probeSubTree, chosenIndex, analysisCtx, true,
-                isLeftOuterJoin, true, context, newNullPlaceHolderVar);
+                isLeftOuterJoin, true, context, newMissingNullPlaceHolderVar, leftOuterMissingValue);
 
         if (indexSearchOp == null) {
             return false;
         }
 
         return AccessMethodUtils.finalizeJoinPlanTransformation(afterJoinRefs, joinRef, indexSubTree, probeSubTree,
-                analysisCtx, context, isLeftOuterJoin, isLeftOuterJoinWithSpecialGroupBy, indexSearchOp,
-                newNullPlaceHolderVar, conditionRef, dataset, chosenIndex);
+                analysisCtx, context, isLeftOuterJoin, isLeftOuterJoinWithSpecialGroupBy, leftOuterMissingValue,
+                indexSearchOp, newMissingNullPlaceHolderVar, conditionRef, dataset, chosenIndex);
     }
 
     /**
@@ -316,7 +317,8 @@
             List<Mutable<ILogicalOperator>> assignBeforeTheOpRefs, OptimizableOperatorSubTree indexSubTree,
             OptimizableOperatorSubTree probeSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
             boolean retainInput, boolean retainMissing, boolean requiresBroadcast, IOptimizationContext context,
-            LogicalVariable newMissingPlaceHolderForLOJ) throws AlgebricksException {
+            LogicalVariable newMissingNullPlaceHolderForLOJ, IAlgebricksConstantValue leftOuterMissingValue)
+            throws AlgebricksException {
 
         Index.ValueIndexDetails chosenIndexDetails = (Index.ValueIndexDetails) chosenIndex.getIndexDetails();
         List<List<String>> chosenIndexKeyFieldNames = chosenIndexDetails.getKeyFieldNames();
@@ -325,8 +327,8 @@
 
         return createBTreeIndexSearchPlan(afterTopOpRefs, topOpRef, conditionRef, assignBeforeTheOpRefs, indexSubTree,
                 probeSubTree, chosenIndex, analysisCtx, retainInput, retainMissing, requiresBroadcast, context,
-                newMissingPlaceHolderForLOJ, chosenIndexKeyFieldNames, chosenIndexKeyFieldTypes,
-                chosenIndexKeyFieldSourceIndicators);
+                newMissingNullPlaceHolderForLOJ, leftOuterMissingValue, chosenIndexKeyFieldNames,
+                chosenIndexKeyFieldTypes, chosenIndexKeyFieldSourceIndicators);
     }
 
     protected ILogicalOperator createBTreeIndexSearchPlan(List<Mutable<ILogicalOperator>> afterTopOpRefs,
@@ -334,9 +336,9 @@
             List<Mutable<ILogicalOperator>> assignBeforeTheOpRefs, OptimizableOperatorSubTree indexSubTree,
             OptimizableOperatorSubTree probeSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
             boolean retainInput, boolean retainMissing, boolean requiresBroadcast, IOptimizationContext context,
-            LogicalVariable newMissingPlaceHolderForLOJ, List<List<String>> chosenIndexKeyFieldNames,
-            List<IAType> chosenIndexKeyFieldTypes, List<Integer> chosenIndexKeyFieldSourceIndicators)
-            throws AlgebricksException {
+            LogicalVariable newMissingNullPlaceHolderForLOJ, IAlgebricksConstantValue leftOuterMissingValue,
+            List<List<String>> chosenIndexKeyFieldNames, List<IAType> chosenIndexKeyFieldTypes,
+            List<Integer> chosenIndexKeyFieldSourceIndicators) throws AlgebricksException {
         Dataset dataset = indexSubTree.getDataset();
         ARecordType recordType = indexSubTree.getRecordType();
         ARecordType metaRecordType = indexSubTree.getMetaRecordType();
@@ -669,7 +671,7 @@
         // The result: SK, PK, [Optional - the result of an instantTrylock on PK]
         ILogicalOperator secondaryIndexUnnestOp = AccessMethodUtils.createSecondaryIndexUnnestMap(dataset, recordType,
                 metaRecordType, chosenIndex, inputOp, jobGenParams, context, retainInput, retainMissing,
-                generateInstantTrylockResultFromIndexSearch);
+                generateInstantTrylockResultFromIndexSearch, leftOuterMissingValue);
 
         // Generate the rest of the upstream plan which feeds the search results into the primary index.
         ILogicalOperator indexSearchOp = null;
@@ -686,7 +688,7 @@
             indexSearchOp = AccessMethodUtils.createRestOfIndexSearchPlan(afterTopOpRefs, topOpRef, conditionRef,
                     assignBeforeTheOpRefs, dataSourceOp, dataset, recordType, metaRecordType, secondaryIndexUnnestOp,
                     context, true, retainInput, retainMissing, false, chosenIndex, analysisCtx, indexSubTree,
-                    probeSubTree, newMissingPlaceHolderForLOJ);
+                    probeSubTree, newMissingNullPlaceHolderForLOJ, leftOuterMissingValue);
 
             // Replaces the datasource scan with the new plan rooted at
             // Get dataSourceRef operator -
@@ -763,7 +765,7 @@
                 } else {
                     unnestMapOp = new LeftOuterUnnestMapOperator(scanVariables,
                             new MutableObject<ILogicalExpression>(primaryIndexSearchFunc), primaryIndexOutputTypes,
-                            true);
+                            leftOuterMissingValue);
                 }
                 unnestMapOp.setSourceLocation(dataSourceOp.getSourceLocation());
                 indexSearchOp = unnestMapOp;
@@ -776,7 +778,7 @@
                 } else {
                     unnestMapOp = new LeftOuterUnnestMapOperator(scanVariables,
                             ((LeftOuterUnnestMapOperator) secondaryIndexUnnestOp).getExpressionRef(),
-                            primaryIndexOutputTypes, true);
+                            primaryIndexOutputTypes, leftOuterMissingValue);
                 }
                 unnestMapOp.setSourceLocation(dataSourceOp.getSourceLocation());
                 indexSearchOp = unnestMapOp;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IAccessMethod.java
index e7becfa..5688a70 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IAccessMethod.java
@@ -32,6 +32,7 @@
 import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
@@ -107,7 +108,8 @@
             List<Mutable<ILogicalOperator>> assignBeforeTheOpRefs, OptimizableOperatorSubTree indexSubTree,
             OptimizableOperatorSubTree probeSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
             boolean retainInput, boolean retainNull, boolean requiresBroadcast, IOptimizationContext context,
-            LogicalVariable newNullPlaceHolderForLOJ) throws AlgebricksException;
+            LogicalVariable newMissingNullPlaceHolderForLOJ, IAlgebricksConstantValue leftOuterMissingValue)
+            throws AlgebricksException;
 
     /**
      * Applies the plan transformation to use chosenIndex to optimize a join query.
@@ -117,8 +119,8 @@
     public boolean applyJoinPlanTransformation(List<Mutable<ILogicalOperator>> afterJoinRefs,
             Mutable<ILogicalOperator> joinRef, OptimizableOperatorSubTree leftSubTree,
             OptimizableOperatorSubTree rightSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
-            IOptimizationContext context, boolean isLeftOuterJoin, boolean isLeftOuterJoinWithSpecialGroupBy)
-            throws AlgebricksException;
+            IOptimizationContext context, boolean isLeftOuterJoin, boolean isLeftOuterJoinWithSpecialGroupBy,
+            IAlgebricksConstantValue leftOuterMissingValue) throws AlgebricksException;
 
     /**
      * Analyzes expr to see whether it is optimizable by the given concrete index.
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
index bc6d21c..6f53219 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
@@ -23,6 +23,7 @@
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 
 import org.apache.asterix.metadata.entities.Dataset;
 import org.apache.asterix.metadata.entities.Index;
@@ -38,6 +39,7 @@
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -389,6 +391,9 @@
                     // Applies the plan transformation using chosen index.
                     AccessMethodAnalysisContext analysisCtx = analyzedAMs.get(chosenIndex.first);
 
+                    IAlgebricksConstantValue leftOuterMissingValue =
+                            isLeftOuterJoin ? ((LeftOuterJoinOperator) joinOp).getMissingValue() : null;
+
                     // For a left outer join with a special GroupBy, prepare objects to reset LOJ's
                     // nullPlaceHolderVariable in that GroupBy's nested plan.
                     // See AccessMethodUtils#removeUnjoinedDuplicatesInLOJ() for a definition of a special GroupBy
@@ -396,14 +401,16 @@
                     boolean isLeftOuterJoinWithSpecialGroupBy;
                     if (isLeftOuterJoin && op.getOperatorTag() == LogicalOperatorTag.GROUP) {
                         GroupByOperator groupByOp = (GroupByOperator) opRef.getValue();
-                        ScalarFunctionCallExpression isNullFuncExpr =
-                                AccessMethodUtils.findLOJIsMissingFuncInGroupBy(groupByOp, rightSubTree);
+                        FunctionIdentifier isMissingNullFuncId = Objects
+                                .requireNonNull(OperatorPropertiesUtil.getIsMissingNullFunction(leftOuterMissingValue));
+                        ScalarFunctionCallExpression isMissingNullFuncExpr = AccessMethodUtils
+                                .findLOJIsMissingNullFuncInGroupBy(groupByOp, rightSubTree, isMissingNullFuncId);
                         // TODO:(dmitry) do we need additional checks to ensure that this is a special GroupBy,
                         // i.e. that this GroupBy will eliminate unjoined duplicates?
-                        isLeftOuterJoinWithSpecialGroupBy = isNullFuncExpr != null;
+                        isLeftOuterJoinWithSpecialGroupBy = isMissingNullFuncExpr != null;
                         if (isLeftOuterJoinWithSpecialGroupBy) {
                             analysisCtx.setLOJSpecialGroupByOpRef(opRef);
-                            analysisCtx.setLOJIsMissingFuncInSpecialGroupBy(isNullFuncExpr);
+                            analysisCtx.setLOJIsMissingNullFuncInSpecialGroupBy(isMissingNullFuncExpr);
                         }
                     } else {
                         isLeftOuterJoinWithSpecialGroupBy = false;
@@ -422,7 +429,7 @@
                     // Finally, tries to apply plan transformation using the chosen index.
                     boolean res = chosenIndex.first.applyJoinPlanTransformation(afterJoinRefs, joinRef, leftSubTree,
                             rightSubTree, chosenIndex.second, analysisCtx, context, isLeftOuterJoin,
-                            isLeftOuterJoinWithSpecialGroupBy);
+                            isLeftOuterJoinWithSpecialGroupBy, leftOuterMissingValue);
 
                     // If the plan transformation is successful, we don't need to traverse the plan
                     // any more, since if there are more JOIN operators, the next trigger on this plan
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroducePrimaryIndexForAggregationRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroducePrimaryIndexForAggregationRule.java
index d221fd1..cdeee0f 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroducePrimaryIndexForAggregationRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroducePrimaryIndexForAggregationRule.java
@@ -44,6 +44,7 @@
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractScanOperator;
@@ -235,7 +236,8 @@
             AbstractUnnestMapOperator primaryIndexUnnestOperator =
                     (AbstractUnnestMapOperator) AccessMethodUtils.createSecondaryIndexUnnestMap(dataset, recordType,
                             metaRecordType, primaryIndex, scanOperator.getInputs().get(0).getValue(),
-                            newBTreeParameters, context, retainInput, false, false);
+                            newBTreeParameters, context, retainInput, false, false,
+                            ConstantExpression.MISSING.getValue());
 
             // re-use the PK variables of the original scan operator
             primaryIndexUnnestOperator.getVariables().clear();
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
index 1027e65..d9b5da9 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
@@ -238,7 +238,7 @@
                     .getExecutionMode() == ExecutionMode.UNPARTITIONED;
             ILogicalOperator subRoot = pair.first.createIndexSearchPlan(afterSelectRefs, selectRef, conditionRef,
                     subTree.getAssignsAndUnnestsRefs(), subTree, null, pair.second, analysisCtx, retainInput, false,
-                    requiresBroadcast, context, null);
+                    requiresBroadcast, context, null, null);
             if (subRoot == null) {
                 return false;
             }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
index a5a2907..ff3d887 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
@@ -65,6 +65,7 @@
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
@@ -400,7 +401,8 @@
             List<Mutable<ILogicalOperator>> assignBeforeTopOpRefs, OptimizableOperatorSubTree indexSubTree,
             OptimizableOperatorSubTree probeSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
             boolean retainInput, boolean retainNull, boolean requiresBroadcast, IOptimizationContext context,
-            LogicalVariable newNullPlaceHolderForLOJ) throws AlgebricksException {
+            LogicalVariable newMissingNullPlaceHolderForLOJ, IAlgebricksConstantValue leftOuterMissingValue)
+            throws AlgebricksException {
         // TODO: we currently do not support the index-only plan for the inverted index searches since
         // there can be many <SK, PK> pairs for the same PK and we may see two different records with the same PK
         // (e.g., the record is deleted and inserted with the same PK). The reason is that there are
@@ -453,13 +455,13 @@
         // since it doesn't contain a field value, only part of it.
         ILogicalOperator secondaryIndexUnnestOp = AccessMethodUtils.createSecondaryIndexUnnestMap(dataset, recordType,
                 metaRecordType, chosenIndex, inputOp, jobGenParams, context, retainInput, retainNull,
-                generateInstantTrylockResultFromIndexSearch);
+                generateInstantTrylockResultFromIndexSearch, leftOuterMissingValue);
 
         // Generates the rest of the upstream plan which feeds the search results into the primary index.
         ILogicalOperator primaryIndexUnnestOp = AccessMethodUtils.createRestOfIndexSearchPlan(afterTopOpRefs, topOpRef,
                 conditionRef, assignBeforeTopOpRefs, dataSourceScan, dataset, recordType, metaRecordType,
                 secondaryIndexUnnestOp, context, true, retainInput, retainNull, false, chosenIndex, analysisCtx,
-                indexSubTree, null, newNullPlaceHolderForLOJ);
+                indexSubTree, null, newMissingNullPlaceHolderForLOJ, leftOuterMissingValue);
 
         return primaryIndexUnnestOp;
     }
@@ -487,10 +489,10 @@
         SelectOperator selectOp = (SelectOperator) selectRef.getValue();
         ILogicalOperator indexPlanRootOp =
                 createIndexSearchPlan(afterSelectRefs, selectRef, selectOp.getCondition(),
-                        subTree.getAssignsAndUnnestsRefs(),
-                        subTree, null, chosenIndex, analysisCtx, false, false, subTree.getDataSourceRef().getValue()
-                                .getInputs().get(0).getValue().getExecutionMode() == ExecutionMode.UNPARTITIONED,
-                        context, null);
+                        subTree.getAssignsAndUnnestsRefs(), subTree,
+                        null, chosenIndex, analysisCtx, false, false, subTree.getDataSourceRef().getValue().getInputs()
+                                .get(0).getValue().getExecutionMode() == ExecutionMode.UNPARTITIONED,
+                        context, null, null);
 
         // Replace the datasource scan with the new plan rooted at primaryIndexUnnestMap.
         subTree.getDataSourceRef().setValue(indexPlanRootOp);
@@ -501,8 +503,8 @@
     public boolean applyJoinPlanTransformation(List<Mutable<ILogicalOperator>> afterJoinRefs,
             Mutable<ILogicalOperator> joinRef, OptimizableOperatorSubTree leftSubTree,
             OptimizableOperatorSubTree rightSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
-            IOptimizationContext context, boolean isLeftOuterJoin, boolean isLeftOuterJoinWithSpecialGroupBy)
-            throws AlgebricksException {
+            IOptimizationContext context, boolean isLeftOuterJoin, boolean isLeftOuterJoinWithSpecialGroupBy,
+            IAlgebricksConstantValue leftOuterMissingValue) throws AlgebricksException {
         Dataset dataset = analysisCtx.getDatasetFromIndexDatasetMap(chosenIndex);
         OptimizableOperatorSubTree indexSubTree;
         OptimizableOperatorSubTree probeSubTree;
@@ -528,16 +530,17 @@
         }
 
         //if LOJ, reset null place holder variable
-        LogicalVariable newNullPlaceHolderVar = null;
+        LogicalVariable newMissingNullPlaceHolderVar = null;
         if (isLeftOuterJoin) {
             //get a new null place holder variable that is the first field variable of the primary key
             //from the indexSubTree's datasourceScanOp
             // We need this for all left outer joins, even those that do not have a special GroupBy
-            newNullPlaceHolderVar = indexSubTree.getDataSourceVariables().get(0);
+            newMissingNullPlaceHolderVar = indexSubTree.getDataSourceVariables().get(0);
 
             if (isLeftOuterJoinWithSpecialGroupBy) {
                 //reset the null place holder variable
-                AccessMethodUtils.resetLOJMissingPlaceholderVarInGroupByOp(analysisCtx, newNullPlaceHolderVar, context);
+                AccessMethodUtils.resetLOJMissingNullPlaceholderVarInGroupByOp(analysisCtx,
+                        newMissingNullPlaceHolderVar, context);
             }
         }
 
@@ -573,14 +576,16 @@
             probeSubTree.setRoot(newProbeRootRef.getValue());
         }
         // Create regular indexed-nested loop join path.
-        ILogicalOperator indexPlanRootOp = createIndexSearchPlan(afterJoinRefs, joinRef,
-                new MutableObject<ILogicalExpression>(joinCond), indexSubTree.getAssignsAndUnnestsRefs(), indexSubTree,
-                probeSubTree, chosenIndex, analysisCtx, true, isLeftOuterJoin, true, context, newNullPlaceHolderVar);
+        ILogicalOperator indexPlanRootOp =
+                createIndexSearchPlan(afterJoinRefs, joinRef, new MutableObject<ILogicalExpression>(joinCond),
+                        indexSubTree.getAssignsAndUnnestsRefs(), indexSubTree, probeSubTree, chosenIndex, analysisCtx,
+                        true, isLeftOuterJoin, true, context, newMissingNullPlaceHolderVar, leftOuterMissingValue);
         indexSubTree.getDataSourceRef().setValue(indexPlanRootOp);
 
         // Change join into a select with the same condition.
-        SelectOperator topSelect = new SelectOperator(new MutableObject<ILogicalExpression>(joinCond), isLeftOuterJoin,
-                newNullPlaceHolderVar);
+        SelectOperator topSelect = isLeftOuterJoin
+                ? new SelectOperator(new MutableObject<>(joinCond), leftOuterMissingValue, newMissingNullPlaceHolderVar)
+                : new SelectOperator(new MutableObject<>(joinCond));
         topSelect.setSourceLocation(indexPlanRootOp.getSourceLocation());
         topSelect.getInputs().add(indexSubTree.getRootRef());
         topSelect.setExecutionMode(ExecutionMode.LOCAL);
@@ -832,7 +837,7 @@
         }
 
         SelectOperator isFilterableSelectOp =
-                new SelectOperator(new MutableObject<ILogicalExpression>(isFilterableExpr), false, null);
+                new SelectOperator(new MutableObject<ILogicalExpression>(isFilterableExpr));
         isFilterableSelectOp.setSourceLocation(sourceLoc);
         isFilterableSelectOp.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
         isFilterableSelectOp.setExecutionMode(ExecutionMode.LOCAL);
@@ -845,7 +850,7 @@
                 FunctionUtil.getFunctionInfo(BuiltinFunctions.NOT), isNotFilterableArgs);
         isNotFilterableExpr.setSourceLocation(sourceLoc);
         SelectOperator isNotFilterableSelectOp =
-                new SelectOperator(new MutableObject<ILogicalExpression>(isNotFilterableExpr), false, null);
+                new SelectOperator(new MutableObject<ILogicalExpression>(isNotFilterableExpr));
         isNotFilterableSelectOp.setSourceLocation(sourceLoc);
         isNotFilterableSelectOp.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
         isNotFilterableSelectOp.setExecutionMode(ExecutionMode.LOCAL);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java
index 73b331c..8d4b83a 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java
@@ -52,6 +52,7 @@
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -190,11 +191,12 @@
 
         analysisCtx.setIndexOnlyPlanInfo(indexOnlyPlanInfo);
 
-        ILogicalOperator primaryIndexUnnestOp = createIndexSearchPlan(afterSelectRefs, selectRef,
-                selectOp.getCondition(), subTree.getAssignsAndUnnestsRefs(), subTree, null, chosenIndex, analysisCtx,
-                AccessMethodUtils.retainInputs(subTree.getDataSourceVariables(), subTree.getDataSourceRef().getValue(),
-                        afterSelectRefs),
-                false, false, context, null);
+        ILogicalOperator primaryIndexUnnestOp =
+                createIndexSearchPlan(afterSelectRefs, selectRef, selectOp.getCondition(),
+                        subTree.getAssignsAndUnnestsRefs(), subTree, null, chosenIndex, analysisCtx,
+                        AccessMethodUtils.retainInputs(subTree.getDataSourceVariables(),
+                                subTree.getDataSourceRef().getValue(), afterSelectRefs),
+                        false, false, context, null, null);
 
         if (primaryIndexUnnestOp == null) {
             return false;
@@ -220,7 +222,8 @@
             List<Mutable<ILogicalOperator>> assignBeforeTopRefs, OptimizableOperatorSubTree indexSubTree,
             OptimizableOperatorSubTree probeSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
             boolean retainInput, boolean retainNull, boolean requiresBroadcast, IOptimizationContext context,
-            LogicalVariable newNullPlaceHolderForLOJ) throws AlgebricksException {
+            LogicalVariable newMissingNullPlaceHolderForLOJ, IAlgebricksConstantValue leftOuterMissingValue)
+            throws AlgebricksException {
         // TODO: We can probably do something smarter here based on selectivity or MBR area.
         IOptimizableFuncExpr optFuncExpr = AccessMethodUtils.chooseFirstOptFuncExpr(chosenIndex, analysisCtx);
 
@@ -302,7 +305,7 @@
 
         ILogicalOperator secondaryIndexUnnestOp = AccessMethodUtils.createSecondaryIndexUnnestMap(dataset, recordType,
                 metaRecordType, chosenIndex, assignSearchKeys, jobGenParams, context, retainInput, retainNull,
-                generateInstantTrylockResultFromIndexSearch);
+                generateInstantTrylockResultFromIndexSearch, leftOuterMissingValue);
 
         // Generates the rest of the upstream plan which feeds the search results into the primary index.
         return dataset.getDatasetType() == DatasetType.EXTERNAL
@@ -311,15 +314,15 @@
                 : AccessMethodUtils.createRestOfIndexSearchPlan(afterTopRefs, topRef, conditionRef, assignBeforeTopRefs,
                         dataSourceOp, dataset, recordType, metaRecordType, secondaryIndexUnnestOp, context, true,
                         retainInput, retainNull, false, chosenIndex, analysisCtx, indexSubTree, null,
-                        newNullPlaceHolderForLOJ);
+                        newMissingNullPlaceHolderForLOJ, leftOuterMissingValue);
     }
 
     @Override
     public boolean applyJoinPlanTransformation(List<Mutable<ILogicalOperator>> afterJoinRefs,
             Mutable<ILogicalOperator> joinRef, OptimizableOperatorSubTree leftSubTree,
             OptimizableOperatorSubTree rightSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
-            IOptimizationContext context, boolean isLeftOuterJoin, boolean isLeftOuterJoinWithSpecialGroupBy)
-            throws AlgebricksException {
+            IOptimizationContext context, boolean isLeftOuterJoin, boolean isLeftOuterJoinWithSpecialGroupBy,
+            IAlgebricksConstantValue leftOuterMissingValue) throws AlgebricksException {
         AbstractBinaryJoinOperator joinOp = (AbstractBinaryJoinOperator) joinRef.getValue();
         Mutable<ILogicalExpression> conditionRef = joinOp.getCondition();
 
@@ -344,12 +347,12 @@
             return false;
         }
 
-        LogicalVariable newNullPlaceHolderVar = null;
+        LogicalVariable newMissingNullPlaceHolderVar = null;
         if (isLeftOuterJoin) {
-            // Gets a new null place holder variable that is the first field variable of the primary key
+            // Gets a new missing (or null) place holder variable that is the first field variable of the primary key
             // from the indexSubTree's datasourceScanOp.
             // We need this for all left outer joins, even those that do not have a special GroupBy
-            newNullPlaceHolderVar = indexSubTree.getDataSourceVariables().get(0);
+            newMissingNullPlaceHolderVar = indexSubTree.getDataSourceVariables().get(0);
         }
 
         boolean canContinue = AccessMethodUtils.setIndexOnlyPlanInfo(afterJoinRefs, joinRef, probeSubTree, indexSubTree,
@@ -360,15 +363,15 @@
 
         ILogicalOperator indexSearchOp = createIndexSearchPlan(afterJoinRefs, joinRef, conditionRef,
                 indexSubTree.getAssignsAndUnnestsRefs(), indexSubTree, probeSubTree, chosenIndex, analysisCtx, true,
-                isLeftOuterJoin, true, context, newNullPlaceHolderVar);
+                isLeftOuterJoin, true, context, newMissingNullPlaceHolderVar, leftOuterMissingValue);
 
         if (indexSearchOp == null) {
             return false;
         }
 
         return AccessMethodUtils.finalizeJoinPlanTransformation(afterJoinRefs, joinRef, indexSubTree, probeSubTree,
-                analysisCtx, context, isLeftOuterJoin, isLeftOuterJoinWithSpecialGroupBy, indexSearchOp,
-                newNullPlaceHolderVar, conditionRef, dataset, chosenIndex);
+                analysisCtx, context, isLeftOuterJoin, isLeftOuterJoinWithSpecialGroupBy, leftOuterMissingValue,
+                indexSearchOp, newMissingNullPlaceHolderVar, conditionRef, dataset, chosenIndex);
     }
 
     @Override
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/AbstractOperatorFromSubplanRewrite.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/AbstractOperatorFromSubplanRewrite.java
index 0f7f354..ba70aff 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/AbstractOperatorFromSubplanRewrite.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/AbstractOperatorFromSubplanRewrite.java
@@ -136,7 +136,7 @@
 
         // Create a copy of this SELECT, and set this to our rewrite root.
         SelectOperator rewriteRootSelect = new SelectOperator(new MutableObject<>(normalizedSelectCondition),
-                optimizableSelect.getRetainMissing(), optimizableSelect.getMissingPlaceholderVariable());
+                optimizableSelect.getRetainMissingAsValue(), optimizableSelect.getMissingPlaceholderVariable());
         rewriteRootSelect.setSourceLocation(sourceLocation);
         rewriteRootSelect.setExecutionMode(optimizableSelect.getExecutionMode());
 
@@ -178,7 +178,8 @@
                     // include this condition.
                     updatedSelectCond = coalesceConditions(rewriteRootSelect, workingOriginalOperator);
                     updatedSelectOperator = new SelectOperator(new MutableObject<>(updatedSelectCond),
-                            rewriteRootSelect.getRetainMissing(), rewriteRootSelect.getMissingPlaceholderVariable());
+                            rewriteRootSelect.getRetainMissingAsValue(),
+                            rewriteRootSelect.getMissingPlaceholderVariable());
                     updatedSelectOperator.setSourceLocation(sourceLocation);
                     updatedSelectOperator.getInputs().addAll(rewriteRootSelect.getInputs());
                     rewriteRootSelect = updatedSelectOperator;
@@ -194,7 +195,7 @@
                     if (traversalOutput != null) {
                         updatedSelectCond = coalesceConditions(rewriteRootSelect, traversalOutput.first);
                         updatedSelectOperator = new SelectOperator(new MutableObject<>(updatedSelectCond),
-                                rewriteRootSelect.getRetainMissing(),
+                                rewriteRootSelect.getRetainMissingAsValue(),
                                 rewriteRootSelect.getMissingPlaceholderVariable());
                         updatedSelectOperator.setSourceLocation(sourceLocation);
                         updatedSelectOperator.getInputs().addAll(rewriteRootSelect.getInputs());
@@ -317,7 +318,7 @@
 
         // First, try to create a SELECT from the aggregate itself (i.e. handle the SOME AND EVERY case).
         if (isNonEmptyStream && aggregateCondition != null) {
-            SelectOperator selectFromAgg = new SelectOperator(new MutableObject<>(aggregateCondition), false, null);
+            SelectOperator selectFromAgg = new SelectOperator(new MutableObject<>(aggregateCondition));
             selectFromAgg.getInputs().addAll(subplanRoot.getInputs());
             selectFromAgg.setSourceLocation(sourceLocation);
             return selectFromAgg;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/MergedSelectRewrite.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/MergedSelectRewrite.java
index 5aefdbf..75168f5 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/MergedSelectRewrite.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/array/MergedSelectRewrite.java
@@ -84,7 +84,7 @@
 
             // Return a new plan that removes all SELECTs that were pushed up.
             SelectOperator newSelectOperator = new SelectOperator(new MutableObject<>(andCond),
-                    originalOperator.getRetainMissing(), originalOperator.getMissingPlaceholderVariable());
+                    originalOperator.getRetainMissingAsValue(), originalOperator.getMissingPlaceholderVariable());
             newSelectOperator.setSourceLocation(originalOperator.getSourceLocation());
             ILogicalPlan newSelectInputPlan = OperatorManipulationUtil
                     .deepCopy(new ALogicalPlanImpl(originalOperator.getInputs().get(0)), context);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/ExpressionValueAccessPushdownVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/ExpressionValueAccessPushdownVisitor.java
index 5616f3f..a8dfe1e 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/ExpressionValueAccessPushdownVisitor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/pushdown/ExpressionValueAccessPushdownVisitor.java
@@ -29,6 +29,7 @@
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
@@ -177,6 +178,6 @@
     private static Set<FunctionIdentifier> createSupportedTypeCheckFunctions() {
         return Set.of(BuiltinFunctions.IS_ARRAY, BuiltinFunctions.IS_OBJECT, BuiltinFunctions.IS_ATOMIC,
                 BuiltinFunctions.IS_NUMBER, BuiltinFunctions.IS_BOOLEAN, BuiltinFunctions.IS_STRING,
-                BuiltinFunctions.IS_MISSING, BuiltinFunctions.IS_NULL, BuiltinFunctions.IS_UNKNOWN);
+                AlgebricksBuiltinFunctions.IS_MISSING, AlgebricksBuiltinFunctions.IS_NULL, BuiltinFunctions.IS_UNKNOWN);
     }
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java
index 8326f85..d4b7853 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java
@@ -21,6 +21,7 @@
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Objects;
 import java.util.Set;
 
 import org.apache.commons.lang3.mutable.Mutable;
@@ -33,6 +34,7 @@
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
 import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
@@ -89,6 +91,7 @@
 
     // The target Nts operator.
     private final ILogicalOperator targetNts;
+    private final IAlgebricksConstantValue leftOuterMissingValue;
 
     // The live variables in <code>subplanInputOperator</code> to enforce.
     private final Set<LogicalVariable> liveVarsFromSubplanInput = new HashSet<>();
@@ -112,13 +115,16 @@
      *            the input operator to the target SubplanOperator
      * @param nts
      *            the NestedTupleSourceOperator to be replaced by <code>subplanInputOperator</code>
+     * @param leftOuterMissingValue
      * @throws AlgebricksException
      */
     public InlineLeftNtsInSubplanJoinFlatteningVisitor(IOptimizationContext context,
-            ILogicalOperator subplanInputOperator, ILogicalOperator nts) throws AlgebricksException {
+            ILogicalOperator subplanInputOperator, ILogicalOperator nts, IAlgebricksConstantValue leftOuterMissingValue)
+            throws AlgebricksException {
         this.context = context;
         this.subplanInputOperator = subplanInputOperator;
         this.targetNts = nts;
+        this.leftOuterMissingValue = Objects.requireNonNull(leftOuterMissingValue);
         VariableUtilities.getSubplanLocalLiveVariables(subplanInputOperator, liveVarsFromSubplanInput);
     }
 
@@ -192,7 +198,7 @@
         AbstractBinaryJoinOperator returnOp = op;
         // After rewriting, the original inner join should become an left outer join.
         if (rewritten) {
-            returnOp = new LeftOuterJoinOperator(op.getCondition());
+            returnOp = new LeftOuterJoinOperator(op.getCondition(), leftOuterMissingValue);
             returnOp.setSourceLocation(op.getSourceLocation());
             returnOp.getInputs().addAll(op.getInputs());
             injectNullCheckVars(returnOp);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineSubplanInputForNestedTupleSourceRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineSubplanInputForNestedTupleSourceRule.java
index ae9a886..9f932a8 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineSubplanInputForNestedTupleSourceRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineSubplanInputForNestedTupleSourceRule.java
@@ -43,6 +43,7 @@
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
 import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
 import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
@@ -59,6 +60,7 @@
 import org.apache.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
 import org.apache.hyracks.algebricks.core.algebra.properties.FunctionalDependency;
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
+import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 import org.apache.hyracks.api.exceptions.SourceLocation;
 
@@ -300,16 +302,19 @@
             return changedAndVarMap;
         }
 
-        /**
+        IAlgebricksConstantValue leftOuterMissingValue = ConstantExpression.MISSING.getValue();
+
+        /*
          * Apply the special join-based rewriting.
          */
-        Pair<Boolean, LinkedHashMap<LogicalVariable, LogicalVariable>> result = applySpecialFlattening(opRef, context);
+        Pair<Boolean, LinkedHashMap<LogicalVariable, LogicalVariable>> result =
+                applySpecialFlattening(opRef, context, leftOuterMissingValue);
         if (!result.first) {
-            /**
+            /*
              * If the special join-based rewriting does not apply, apply the general
              * rewriting which blindly inlines all NTSs.
              */
-            result = applyGeneralFlattening(opRef, context);
+            result = applyGeneralFlattening(opRef, context, leftOuterMissingValue);
         }
         LinkedHashMap<LogicalVariable, LogicalVariable> returnedMap = new LinkedHashMap<>();
         // Adds variable mappings from input operators.
@@ -359,7 +364,8 @@
     }
 
     private Pair<Boolean, LinkedHashMap<LogicalVariable, LogicalVariable>> applyGeneralFlattening(
-            Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+            Mutable<ILogicalOperator> opRef, IOptimizationContext context,
+            IAlgebricksConstantValue leftOuterMissingValue) throws AlgebricksException {
         SubplanOperator subplanOp = (SubplanOperator) opRef.getValue();
         if (!SubplanFlatteningUtil.containsOperators(subplanOp, EnumSet.of(LogicalOperatorTag.DATASOURCESCAN,
                 LogicalOperatorTag.INNERJOIN, LogicalOperatorTag.LEFTOUTERJOIN))) {
@@ -437,8 +443,8 @@
         } else {
             joinExpr = joinPredicates.size() > 0 ? joinPredicates.get(0).getValue() : ConstantExpression.TRUE;
         }
-        LeftOuterJoinOperator leftOuterJoinOp =
-                new LeftOuterJoinOperator(new MutableObject<>(joinExpr), inputOpRef, rightInputOpRef);
+        LeftOuterJoinOperator leftOuterJoinOp = new LeftOuterJoinOperator(new MutableObject<>(joinExpr), inputOpRef,
+                rightInputOpRef, leftOuterMissingValue);
         leftOuterJoinOp.setSourceLocation(sourceLoc);
         OperatorManipulationUtil.computeTypeEnvironmentBottomUp(rightInputOp, context);
         context.computeAndSetTypeEnvironmentForOperator(leftOuterJoinOp);
@@ -481,7 +487,7 @@
             lowestAggregateRefInSubplan.getValue().getInputs().add(currentOpRef);
         }
 
-        // Adds a select operator into the nested plan for group-by to remove tuples with NULL on {@code assignVar},
+        // Adds a select operator into the nested plan for group-by to remove tuples with MISSING on {@code assignVar},
         // i.e., subplan input tuples that are filtered out within a subplan.
         VariableReferenceExpression assignVarRef = new VariableReferenceExpression(assignVar);
         assignVarRef.setSourceLocation(sourceLoc);
@@ -489,14 +495,15 @@
         List<Mutable<ILogicalExpression>> args = new ArrayList<>();
         args.add(filterVarExpr);
         List<Mutable<ILogicalExpression>> argsForNotFunction = new ArrayList<>();
-        ScalarFunctionCallExpression isMissingExpr =
-                new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.IS_MISSING), args);
+        ScalarFunctionCallExpression isMissingExpr = new ScalarFunctionCallExpression(
+                FunctionUtil.getFunctionInfo(OperatorPropertiesUtil.getIsMissingNullFunction(leftOuterMissingValue)),
+                args);
         isMissingExpr.setSourceLocation(sourceLoc);
         argsForNotFunction.add(new MutableObject<>(isMissingExpr));
         ScalarFunctionCallExpression notExpr = new ScalarFunctionCallExpression(
                 FunctionUtil.getFunctionInfo(BuiltinFunctions.NOT), argsForNotFunction);
         notExpr.setSourceLocation(sourceLoc);
-        SelectOperator selectOp = new SelectOperator(new MutableObject<>(notExpr), false, null);
+        SelectOperator selectOp = new SelectOperator(new MutableObject<>(notExpr));
         selectOp.setSourceLocation(sourceLoc);
         currentOpRef.getValue().getInputs().add(new MutableObject<>(selectOp));
 
@@ -524,7 +531,8 @@
     }
 
     private Pair<Boolean, LinkedHashMap<LogicalVariable, LogicalVariable>> applySpecialFlattening(
-            Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+            Mutable<ILogicalOperator> opRef, IOptimizationContext context,
+            IAlgebricksConstantValue leftOuterMissingValue) throws AlgebricksException {
         SubplanOperator subplanOp = (SubplanOperator) opRef.getValue();
         SourceLocation sourceLoc = subplanOp.getSourceLocation();
         Mutable<ILogicalOperator> inputOpRef = subplanOp.getInputs().get(0);
@@ -554,8 +562,8 @@
         Set<LogicalVariable> liveVars = new HashSet<>();
         VariableUtilities.getLiveVariables(inputOp, liveVars);
 
-        Pair<Set<LogicalVariable>, Mutable<ILogicalOperator>> notNullVarsAndTopJoinRef =
-                SubplanFlatteningUtil.inlineLeftNtsInSubplanJoin(subplanOp, context, newPrimaryKeyFd);
+        Pair<Set<LogicalVariable>, Mutable<ILogicalOperator>> notNullVarsAndTopJoinRef = SubplanFlatteningUtil
+                .inlineLeftNtsInSubplanJoin(subplanOp, context, newPrimaryKeyFd, leftOuterMissingValue);
         if (notNullVarsAndTopJoinRef.first == null) {
             inputOpRef.setValue(inputOpBackup);
             return new Pair<>(false, replacedVarMap);
@@ -588,7 +596,7 @@
         groupbyOp.getInputs().add(new MutableObject<>(topJoinRef.getValue()));
 
         if (!notNullVars.isEmpty()) {
-            // Adds a select operator into the nested plan for group-by to remove tuples with NULL on {@code assignVar},
+            // Adds a select operator into the nested plan for group-by to remove tuples with MISSING on {@code assignVar},
             // i.e., subplan input tuples that are filtered out within a subplan.
             List<Mutable<ILogicalExpression>> nullCheckExprRefs = new ArrayList<>();
             for (LogicalVariable notNullVar : notNullVars) {
@@ -598,8 +606,8 @@
                 List<Mutable<ILogicalExpression>> args = new ArrayList<>();
                 args.add(filterVarExpr);
                 List<Mutable<ILogicalExpression>> argsForNotFunction = new ArrayList<>();
-                ScalarFunctionCallExpression isMissingExpr = new ScalarFunctionCallExpression(
-                        FunctionUtil.getFunctionInfo(BuiltinFunctions.IS_MISSING), args);
+                ScalarFunctionCallExpression isMissingExpr = new ScalarFunctionCallExpression(FunctionUtil
+                        .getFunctionInfo(OperatorPropertiesUtil.getIsMissingNullFunction(leftOuterMissingValue)), args);
                 isMissingExpr.setSourceLocation(sourceLoc);
                 argsForNotFunction.add(new MutableObject<>(isMissingExpr));
                 ScalarFunctionCallExpression notExpr = new ScalarFunctionCallExpression(
@@ -616,7 +624,7 @@
             } else {
                 selectExprRef = nullCheckExprRefs.get(0);
             }
-            SelectOperator selectOp = new SelectOperator(selectExprRef, false, null);
+            SelectOperator selectOp = new SelectOperator(selectExprRef);
             selectOp.setSourceLocation(sourceLoc);
             topJoinRef.setValue(selectOp);
             NestedTupleSourceOperator ntsOp = new NestedTupleSourceOperator(new MutableObject<>(groupbyOp));
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanFlatteningUtil.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanFlatteningUtil.java
index e3c7d69..916d75d 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanFlatteningUtil.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanFlatteningUtil.java
@@ -32,6 +32,7 @@
 import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
@@ -101,13 +102,14 @@
      *            the optimization context
      * @param extraPrimaryKeyFd
      *            extra primary key dependency that needs to be added to the context before performing the rewrite
+     * @param leftOuterMissingValue
      * @return A set of variables used for further null-checks, i.e., variables indicating
      *         whether a tuple produced by a transformed left outer join is a non-match;
      *         a reference to the top join operator in the nested subplan.
      */
     public static Pair<Set<LogicalVariable>, Mutable<ILogicalOperator>> inlineLeftNtsInSubplanJoin(
-            SubplanOperator subplanOp, IOptimizationContext context, FunctionalDependency extraPrimaryKeyFd)
-            throws AlgebricksException {
+            SubplanOperator subplanOp, IOptimizationContext context, FunctionalDependency extraPrimaryKeyFd,
+            IAlgebricksConstantValue leftOuterMissingValue) throws AlgebricksException {
         Pair<Boolean, ILogicalOperator> applicableAndNtsToRewrite =
                 SubplanFlatteningUtil.isQualifiedForSpecialFlattening(subplanOp);
         if (!applicableAndNtsToRewrite.first) {
@@ -120,8 +122,8 @@
 
         ILogicalOperator qualifiedNts = applicableAndNtsToRewrite.second;
         ILogicalOperator subplanInputOp = subplanOp.getInputs().get(0).getValue();
-        InlineLeftNtsInSubplanJoinFlatteningVisitor specialVisitor =
-                new InlineLeftNtsInSubplanJoinFlatteningVisitor(context, subplanInputOp, qualifiedNts);
+        InlineLeftNtsInSubplanJoinFlatteningVisitor specialVisitor = new InlineLeftNtsInSubplanJoinFlatteningVisitor(
+                context, subplanInputOp, qualifiedNts, leftOuterMissingValue);
 
         // Rewrites the query plan.
         Mutable<ILogicalOperator> topRef = subplanOp.getNestedPlans().get(0).getRoots().get(0);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
index d664da1..794a0b7 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
@@ -1372,7 +1372,7 @@
 
         switch (qe.getQuantifier()) {
             case SOME:
-                SelectOperator s = new SelectOperator(new MutableObject<>(eo2.first), false, null);
+                SelectOperator s = new SelectOperator(new MutableObject<>(eo2.first));
                 s.getInputs().add(eo2.second);
                 s.setSourceLocation(sourceLoc);
                 AggregateFunctionCallExpression fAgg = BuiltinFunctions
@@ -1400,7 +1400,7 @@
                 ScalarFunctionCallExpression notExpr = new ScalarFunctionCallExpression(
                         BuiltinFunctions.getBuiltinFunctionInfo(AlgebricksBuiltinFunctions.NOT), notArgs);
                 notExpr.setSourceLocation(sourceLoc);
-                s = new SelectOperator(new MutableObject<>(notExpr), false, null);
+                s = new SelectOperator(new MutableObject<>(notExpr));
                 s.getInputs().add(eo2.second);
                 s.setSourceLocation(sourceLoc);
                 fAgg = BuiltinFunctions.makeAggregateFunctionExpression(BuiltinFunctions.EMPTY_STREAM,
@@ -1570,7 +1570,7 @@
     public Pair<ILogicalOperator, LogicalVariable> visit(WhereClause w, Mutable<ILogicalOperator> tupSource)
             throws CompilationException {
         Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = langExprToAlgExpression(w.getWhereExpr(), tupSource);
-        SelectOperator s = new SelectOperator(new MutableObject<>(p.first), false, null);
+        SelectOperator s = new SelectOperator(new MutableObject<>(p.first));
         s.getInputs().add(p.second);
         s.setSourceLocation(w.getSourceLocation());
         return new Pair<>(s, null);
@@ -2024,7 +2024,7 @@
         subplanOp.setSourceLocation(sourceLoc);
         NestedTupleSourceOperator ntsOp = new NestedTupleSourceOperator(new MutableObject<>(subplanOp));
         ntsOp.setSourceLocation(sourceLoc);
-        SelectOperator select = new SelectOperator(selectExpr, false, null);
+        SelectOperator select = new SelectOperator(selectExpr);
         // The select operator cannot be moved up and down, otherwise it will cause
         // typing issues (ASTERIXDB-1203).
         OperatorPropertiesUtil.markMovable(select, false);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java
index 2dd4f96..65b0f03 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java
@@ -112,6 +112,7 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AggregateFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionAnnotation;
 import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
@@ -347,7 +348,7 @@
         if (joinClause.getJoinType() == JoinType.INNER) {
             Mutable<ILogicalOperator> rightInputRef = uncorrelatedRightBranchStack.peek();
             Pair<ILogicalOperator, LogicalVariable> rightBranch =
-                    generateUnnestForBinaryCorrelateRightBranch(joinClause, rightInputRef, true);
+                    generateUnnestForBinaryCorrelateRightBranch(joinClause, rightInputRef, false, null);
             // A join operator with condition TRUE.
             AbstractBinaryJoinOperator joinOperator = new InnerJoinOperator(
                     new MutableObject<>(ConstantExpression.TRUE), leftInputRef, new MutableObject<>(rightBranch.first));
@@ -357,7 +358,7 @@
             // Add an additional filter operator.
             Pair<ILogicalExpression, Mutable<ILogicalOperator>> conditionExprOpPair =
                     langExprToAlgExpression(joinClause.getConditionExpression(), joinOpRef);
-            SelectOperator filter = new SelectOperator(new MutableObject<>(conditionExprOpPair.first), false, null);
+            SelectOperator filter = new SelectOperator(new MutableObject<>(conditionExprOpPair.first));
             filter.getInputs().add(conditionExprOpPair.second);
             filter.setSourceLocation(conditionExprOpPair.first.getSourceLocation());
             return new Pair<>(filter, rightBranch.second);
@@ -375,13 +376,13 @@
 
             // Adds an unnest operator to unnest to right expression.
             Pair<ILogicalOperator, LogicalVariable> rightBranch =
-                    generateUnnestForBinaryCorrelateRightBranch(joinClause, ntsRef, true);
+                    generateUnnestForBinaryCorrelateRightBranch(joinClause, ntsRef, false, null);
             AbstractUnnestNonMapOperator rightUnnestOp = (AbstractUnnestNonMapOperator) rightBranch.first;
 
             // Adds an additional filter operator for the join condition.
             Pair<ILogicalExpression, Mutable<ILogicalOperator>> conditionExprOpPair =
                     langExprToAlgExpression(joinClause.getConditionExpression(), new MutableObject<>(rightUnnestOp));
-            SelectOperator filter = new SelectOperator(new MutableObject<>(conditionExprOpPair.first), false, null);
+            SelectOperator filter = new SelectOperator(new MutableObject<>(conditionExprOpPair.first));
             filter.getInputs().add(conditionExprOpPair.second);
             filter.setSourceLocation(conditionExprOpPair.first.getSourceLocation());
 
@@ -448,7 +449,8 @@
             Pair<ILogicalExpression, Mutable<ILogicalOperator>> pUnnestExpr =
                     makeUnnestExpression(aggVarRefExpr, new MutableObject<>(subplanOp));
             LeftOuterUnnestOperator outerUnnestOp =
-                    new LeftOuterUnnestOperator(outerUnnestVar, new MutableObject<>(pUnnestExpr.first));
+                    new LeftOuterUnnestOperator(outerUnnestVar, new MutableObject<>(pUnnestExpr.first),
+                            translateLeftOuterMissingValue(joinClause.getOuterJoinMissingValueType()));
             outerUnnestOp.getInputs().add(pUnnestExpr.second);
             outerUnnestOp.setSourceLocation(aggOp.getSourceLocation());
             currentTopOp = outerUnnestOp;
@@ -506,6 +508,18 @@
         }
     }
 
+    private static IAlgebricksConstantValue translateLeftOuterMissingValue(Literal.Type type)
+            throws CompilationException {
+        switch (type) {
+            case MISSING:
+                return ConstantExpression.MISSING.getValue();
+            case NULL:
+                return ConstantExpression.NULL.getValue();
+            default:
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, String.valueOf(type));
+        }
+    }
+
     @Override
     public Pair<ILogicalOperator, LogicalVariable> visit(NestClause nestClause, Mutable<ILogicalOperator> arg)
             throws CompilationException {
@@ -516,8 +530,9 @@
     @Override
     public Pair<ILogicalOperator, LogicalVariable> visit(UnnestClause unnestClause,
             Mutable<ILogicalOperator> inputOpRef) throws CompilationException {
-        return generateUnnestForBinaryCorrelateRightBranch(unnestClause, inputOpRef,
-                unnestClause.getUnnestType() == UnnestType.INNER);
+        boolean outerUnnest = unnestClause.getUnnestType() == UnnestType.LEFTOUTER;
+        return generateUnnestForBinaryCorrelateRightBranch(unnestClause, inputOpRef, outerUnnest,
+                outerUnnest ? translateLeftOuterMissingValue(unnestClause.getOuterUnnestMissingValueType()) : null);
     }
 
     @Override
@@ -525,14 +540,14 @@
             throws CompilationException {
         Pair<ILogicalExpression, Mutable<ILogicalOperator>> p =
                 langExprToAlgExpression(havingClause.getFilterExpression(), tupSource);
-        SelectOperator s = new SelectOperator(new MutableObject<>(p.first), false, null);
+        SelectOperator s = new SelectOperator(new MutableObject<>(p.first));
         s.getInputs().add(p.second);
         return new Pair<>(s, null);
     }
 
     private Pair<ILogicalOperator, LogicalVariable> generateUnnestForBinaryCorrelateRightBranch(
-            AbstractBinaryCorrelateClause binaryCorrelate, Mutable<ILogicalOperator> inputOpRef, boolean innerUnnest)
-            throws CompilationException {
+            AbstractBinaryCorrelateClause binaryCorrelate, Mutable<ILogicalOperator> inputOpRef, boolean outerUnnest,
+            IAlgebricksConstantValue outerUnnestMissingValue) throws CompilationException {
         LogicalVariable rightVar = context.newVarFromExpression(binaryCorrelate.getRightVariable());
         Expression rightExpr = binaryCorrelate.getRightExpression();
         Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = langExprToAlgExpression(rightExpr, inputOpRef);
@@ -541,13 +556,15 @@
         if (binaryCorrelate.hasPositionalVariable()) {
             LogicalVariable pVar = context.newVarFromExpression(binaryCorrelate.getPositionalVariable());
             // We set the positional variable type as BIGINT type.
-            unnestOp = innerUnnest
-                    ? new UnnestOperator(rightVar, new MutableObject<>(pUnnestExpr.first), pVar, BuiltinType.AINT64)
-                    : new LeftOuterUnnestOperator(rightVar, new MutableObject<>(pUnnestExpr.first), pVar,
-                            BuiltinType.AINT64);
+            unnestOp = outerUnnest
+                    ? new LeftOuterUnnestOperator(rightVar, new MutableObject<>(pUnnestExpr.first), pVar,
+                            BuiltinType.AINT64, outerUnnestMissingValue)
+                    : new UnnestOperator(rightVar, new MutableObject<>(pUnnestExpr.first), pVar, BuiltinType.AINT64);
         } else {
-            unnestOp = innerUnnest ? new UnnestOperator(rightVar, new MutableObject<>(pUnnestExpr.first))
-                    : new LeftOuterUnnestOperator(rightVar, new MutableObject<>(pUnnestExpr.first));
+            unnestOp = outerUnnest
+                    ? new LeftOuterUnnestOperator(rightVar, new MutableObject<>(pUnnestExpr.first),
+                            outerUnnestMissingValue)
+                    : new UnnestOperator(rightVar, new MutableObject<>(pUnnestExpr.first));
         }
         unnestOp.getInputs().add(pUnnestExpr.second);
         unnestOp.setSourceLocation(binaryCorrelate.getRightVariable().getSourceLocation());
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
index f2bb66d..3a9d54d 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
@@ -320,6 +320,7 @@
         builder.setHashFunctionFactoryProvider(format.getBinaryHashFunctionFactoryProvider());
         builder.setHashFunctionFamilyProvider(format.getBinaryHashFunctionFamilyProvider());
         builder.setMissingWriterFactory(format.getMissingWriterFactory());
+        builder.setNullWriterFactory(format.getNullWriterFactory());
         builder.setUnnestingPositionWriterFactory(format.getUnnestingPositionWriterFactory());
         builder.setPredicateEvaluatorFactoryProvider(format.getPredicateEvaluatorFactoryProvider());
         builder.setPrinterProvider(getPrinterFactoryProvider(format, conf.fmt()));
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
index d4e9cf8..f40628fb 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
@@ -439,7 +439,7 @@
                 storageComponentProvider.getStorageManager(), primaryIndexInfo.getFileSplitProvider());
         BTreeSearchOperatorDescriptor searchOpDesc = new BTreeSearchOperatorDescriptor(spec, primaryIndexInfo.rDesc,
                 null, null, true, true, indexDataflowHelperFactory, false, false, null,
-                NoOpOperationCallbackFactory.INSTANCE, filterFields, filterFields, false);
+                NoOpOperationCallbackFactory.INSTANCE, filterFields, filterFields, false, null);
         BTreeSearchOperatorNodePushable searchOp =
                 searchOpDesc.createPushRuntime(ctx, primaryIndexInfo.getSearchRecordDescriptorProvider(),
                         ctx.getTaskAttemptId().getTaskId().getPartition(), 1);
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/resource/PlanStagesGeneratorTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/resource/PlanStagesGeneratorTest.java
index 8969f18..b0de85e 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/resource/PlanStagesGeneratorTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/resource/PlanStagesGeneratorTest.java
@@ -210,7 +210,8 @@
         exchangeOperator2.setPhysicalOperator(new HashPartitionExchangePOperator(Collections.emptyList(), null));
         exchangeOperator2.getInputs().add(new MutableObject<>(groupByOperator));
 
-        LeftOuterJoinOperator secondJoin = new LeftOuterJoinOperator(new MutableObject<>(ConstantExpression.TRUE));
+        LeftOuterJoinOperator secondJoin = new LeftOuterJoinOperator(new MutableObject<>(ConstantExpression.TRUE),
+                ConstantExpression.MISSING.getValue());
         secondJoin.setExecutionMode(PARTITIONED);
         secondJoin.setPhysicalOperator(new NestedLoopJoinPOperator(secondJoin.getJoinKind(),
                 AbstractJoinPOperator.JoinPartitioningType.BROADCAST));
@@ -270,7 +271,8 @@
         order2.setPhysicalOperator(new StableSortPOperator());
         order2.getInputs().add(new MutableObject<>(replicateOperator));
 
-        LeftOuterJoinOperator secondJoin = new LeftOuterJoinOperator(new MutableObject<>(ConstantExpression.TRUE));
+        LeftOuterJoinOperator secondJoin = new LeftOuterJoinOperator(new MutableObject<>(ConstantExpression.TRUE),
+                ConstantExpression.MISSING.getValue());
         secondJoin.setExecutionMode(PARTITIONED);
         secondJoin.setPhysicalOperator(new NestedLoopJoinPOperator(secondJoin.getJoinKind(),
                 AbstractJoinPOperator.JoinPartitioningType.BROADCAST));
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppRQGGroupingSetsIT.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppRQGGroupingSetsIT.java
index 2ccab70..e9db5fa 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppRQGGroupingSetsIT.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppRQGGroupingSetsIT.java
@@ -102,7 +102,7 @@
 
     @Test
     public void test() throws Exception {
-        runTestCase(testcaseId, groupByClause, sqlQuery, sqlppQuery);
+        runTestCase(testcaseId, groupByClause, sqlQuery, sqlppQuery, null);
     }
 
     private static TestQuery generateQuery(int testcaseId, Random random) {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppRQGJoinsIT.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppRQGJoinsIT.java
index 96e188e..7dfd0f0 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppRQGJoinsIT.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppRQGJoinsIT.java
@@ -21,9 +21,13 @@
 
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import java.util.function.IntUnaryOperator;
 
+import org.apache.asterix.api.http.server.QueryServiceRequestParameters;
+import org.apache.asterix.testframework.xml.ParameterTypeEnum;
+import org.apache.asterix.testframework.xml.TestCase;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -39,7 +43,9 @@
 
     private final String sqlppQuery;
 
-    private final String desc;
+    private final boolean sqlCompatMode;
+
+    private final String description;
 
     static final String PROJECT_FIELD = "unique1";
 
@@ -49,31 +55,34 @@
 
     static final char[] SHAPES = new char[] { 'c', 's', 'q' };
 
-    @Parameterized.Parameters(name = "SqlppRQGJoinsIT {index}: {3}")
+    @Parameterized.Parameters(name = "SqlppRQGJoinsIT {index}: {1}")
     public static Collection<Object[]> tests() {
         List<Object[]> testCases = new ArrayList<>();
 
         IntUnaryOperator filterComputer = i -> 2 * (i + 1);
 
+        boolean[] allSqlCompatModes = new boolean[] { false, true };
         String[] allJoinKinds = new String[] { "INNER", "LEFT", "RIGHT" };
         String[] queryJoinKinds = new String[3];
         int id = 0;
 
-        for (String jk0 : allJoinKinds) {
-            queryJoinKinds[0] = jk0;
-            TestQuery q1 = generateQuery(queryJoinKinds, 1, filterComputer, SHAPES[0]);
-            testCases.add(new Object[] { id++, q1.sqlQuery, q1.sqlppQuery, q1.summary });
+        for (boolean sqlCompatMode : allSqlCompatModes) {
+            for (String jk0 : allJoinKinds) {
+                queryJoinKinds[0] = jk0;
+                TestQuery q1 = generateQuery(queryJoinKinds, 1, filterComputer, SHAPES[0], sqlCompatMode);
+                addTestCase(testCases, id++, q1);
 
-            for (char s : SHAPES) {
-                for (String jk1 : allJoinKinds) {
-                    queryJoinKinds[1] = jk1;
-                    TestQuery q2 = generateQuery(queryJoinKinds, 2, filterComputer, s);
-                    testCases.add(new Object[] { id++, q2.sqlQuery, q2.sqlppQuery, q2.summary });
+                for (char s : SHAPES) {
+                    for (String jk1 : allJoinKinds) {
+                        queryJoinKinds[1] = jk1;
+                        TestQuery q2 = generateQuery(queryJoinKinds, 2, filterComputer, s, sqlCompatMode);
+                        addTestCase(testCases, id++, q2);
 
-                    for (String jk2 : allJoinKinds) {
-                        queryJoinKinds[2] = jk2;
-                        TestQuery q3 = generateQuery(queryJoinKinds, 3, filterComputer, s);
-                        testCases.add(new Object[] { id++, q3.sqlQuery, q3.sqlppQuery, q3.summary });
+                        for (String jk2 : allJoinKinds) {
+                            queryJoinKinds[2] = jk2;
+                            TestQuery q3 = generateQuery(queryJoinKinds, 3, filterComputer, s, sqlCompatMode);
+                            addTestCase(testCases, id++, q3);
+                        }
                     }
                 }
             }
@@ -83,7 +92,7 @@
     }
 
     private static TestQuery generateQuery(String[] joinKinds, int joinKindsSize, IntUnaryOperator filterComputer,
-            char shape) {
+            char shape, boolean sqlCompatMode) {
         int tCount = joinKindsSize + 1;
         List<String> tDefs = new ArrayList<>(tCount);
         for (int i = 0; i < tCount; i++) {
@@ -175,7 +184,7 @@
                 summary.append(joinKinds[i - 1]);
             }
             String projectFieldExprSql = String.format(fieldExprFormat, tThis, PROJECT_FIELD);
-            String projectFieldExprSqlpp = missing2Null(projectFieldExprSql);
+            String projectFieldExprSqlpp = sqlCompatMode ? projectFieldExprSql : missing2Null(projectFieldExprSql);
             String projectFieldAlias = String.format("%s_%s", tThis, PROJECT_FIELD);
             String projectFormat = "%s AS %s";
             selectClauseSql.append(String.format(projectFormat, projectFieldExprSql, projectFieldAlias));
@@ -188,22 +197,32 @@
             summary.append(';').append(shape);
         }
 
+        if (sqlCompatMode) {
+            summary.append(";sql-compat");
+        }
+
         String queryFormat = "SELECT %s %sORDER BY %s";
         String sqlQuery = String.format(queryFormat, selectClauseSql, fromClauseSql, orderbyClauseSql);
         String sqlppQuery = String.format(queryFormat, selectClauseSqlpp, fromClauseSqlpp, orderbyClauseSqlpp);
 
-        return new TestQuery(sqlQuery, sqlppQuery, summary.toString());
+        return new TestQuery(sqlQuery, sqlppQuery, sqlCompatMode, summary.toString());
     }
 
     private static String missing2Null(String expr) {
         return String.format("if_missing(%s, null)", expr);
     }
 
-    public SqlppRQGJoinsIT(int testcaseId, String sqlQuery, String sqlppQuery, String desc) {
+    private static void addTestCase(List<Object[]> testCases, int id, TestQuery q) {
+        testCases.add(new Object[] { id, q.summary, q.sqlQuery, q.sqlppQuery, q.sqlCompatMode });
+    }
+
+    public SqlppRQGJoinsIT(int testcaseId, String description, String sqlQuery, String sqlppQuery,
+            boolean sqlCompatMode) {
         this.testcaseId = testcaseId;
+        this.description = description;
         this.sqlQuery = sqlQuery;
         this.sqlppQuery = sqlppQuery;
-        this.desc = desc;
+        this.sqlCompatMode = sqlCompatMode;
     }
 
     @BeforeClass
@@ -218,17 +237,27 @@
 
     @Test
     public void test() throws Exception {
-        runTestCase(testcaseId, desc, sqlQuery, sqlppQuery);
+        List<TestCase.CompilationUnit.Parameter> params = null;
+        if (sqlCompatMode) {
+            TestCase.CompilationUnit.Parameter sqlCompatModeParam = new TestCase.CompilationUnit.Parameter();
+            sqlCompatModeParam.setName(QueryServiceRequestParameters.Parameter.SQL_COMPAT.str());
+            sqlCompatModeParam.setType(ParameterTypeEnum.JSON);
+            sqlCompatModeParam.setValue(String.valueOf(sqlCompatMode));
+            params = Collections.singletonList(sqlCompatModeParam);
+        }
+        runTestCase(testcaseId, description, sqlQuery, sqlppQuery, params);
     }
 
     private static class TestQuery {
         final String sqlQuery;
         final String sqlppQuery;
         final String summary;
+        boolean sqlCompatMode;
 
-        TestQuery(String sqlQuery, String sqlppQuery, String summary) {
+        TestQuery(String sqlQuery, String sqlppQuery, boolean sqlCompatMode, String summary) {
             this.sqlQuery = sqlQuery;
             this.sqlppQuery = sqlppQuery;
+            this.sqlCompatMode = sqlCompatMode;
             this.summary = summary;
         }
     }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppRQGTestBase.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppRQGTestBase.java
index 50f2b93..aa8783a 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppRQGTestBase.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppRQGTestBase.java
@@ -25,6 +25,7 @@
 import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.io.PrintWriter;
+import java.net.URI;
 import java.nio.charset.StandardCharsets;
 import java.nio.file.Files;
 import java.nio.file.Path;
@@ -54,6 +55,7 @@
 import org.apache.asterix.test.common.TestExecutor;
 import org.apache.asterix.test.common.TestHelper;
 import org.apache.asterix.testframework.context.TestCaseContext;
+import org.apache.asterix.testframework.xml.TestCase;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.logging.log4j.LogManager;
@@ -141,8 +143,8 @@
         return result;
     }
 
-    protected void runTestCase(int testcaseId, String testcaseDescription, String sqlQuery, String sqlppQuery)
-            throws Exception {
+    protected void runTestCase(int testcaseId, String testcaseDescription, String sqlQuery, String sqlppQuery,
+            List<TestCase.CompilationUnit.Parameter> params) throws Exception {
         LOGGER.info(String.format("Starting testcase #%d: %s", testcaseId, testcaseDescription));
 
         LOGGER.info("Running SQL");
@@ -156,8 +158,9 @@
         LOGGER.info("Running SQL++");
         LOGGER.info(sqlppQuery);
         ArrayNode sqlppResult;
-        try (InputStream resultStream = testExecutor.executeQueryService(sqlppQuery,
-                testExecutor.getEndpoint(Servlets.QUERY_SERVICE), TestCaseContext.OutputFormat.ADM)) {
+        URI endpoint = testExecutor.getEndpoint(Servlets.QUERY_SERVICE);
+        try (InputStream resultStream = testExecutor.executeQueryService(sqlppQuery, TestCaseContext.OutputFormat.ADM,
+                endpoint, params == null ? Collections.emptyList() : params, false, StandardCharsets.UTF_8)) {
             sqlppResult = asJson(
                     ResultExtractor.extract(resultStream, StandardCharsets.UTF_8, TestCaseContext.OutputFormat.ADM));
         }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.1.ddl.sqlpp
new file mode 100644
index 0000000..909b74c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.1.ddl.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/* Test that outer join emits NULLs for non-matching values */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create dataset ds1(id int) open type primary key id;
+
+create dataset ds2(id int) open type primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.2.update.sqlpp
new file mode 100644
index 0000000..65e9d54
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.2.update.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into ds1 ({"id": 1, "a": 1, "b": 1 });
+insert into ds1 ({"id": 2, "a": 1, "b": 2 });
+insert into ds1 ({"id": 3, "a": 2, "b": 3 });
+insert into ds1 ({"id": 4, "a": 2, "b": 4 });
+insert into ds1 ({"id": 5, "a": 3, "b": 5 });
+insert into ds1 ({"id": 6, "a": 3, "b": 6 });
+insert into ds1 ({"id": 7, "a": 4, "b": 7 });
+
+insert into ds2 ({"id": 100, "x": 2, "y": 100 });
+insert into ds2 ({"id": 101, "x": 2, "y": 101 });
+insert into ds2 ({"id": 102, "x": 3, "y": 102 });
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.3.query.sqlpp
new file mode 100644
index 0000000..c2edfe6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.3.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+/* Left outer join / Hash */
+
+use test;
+
+select ds1.b, ds2.y
+from ds1 left outer join ds2 on ds1.a = ds2.x
+order by ds1.b, ds2.y;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.4.query.sqlpp
new file mode 100644
index 0000000..3f85f0c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.4.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+/* Right outer join / Hash */
+
+use test;
+
+select ds1.b, ds2.y
+from ds2 right outer join ds1 on ds1.a = ds2.x
+order by ds1.b, ds2.y;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.5.query.sqlpp
new file mode 100644
index 0000000..6ae5a89
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.5.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+/* Left outer join / NL */
+
+use test;
+
+select ds1.b, ds2.y
+from ds1 left outer join ds2 on (ds1.a - ds2.x >= 0 and ds2.x - ds1.a >= 0)
+order by ds1.b, ds2.y;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.6.query.sqlpp
new file mode 100644
index 0000000..e4a5ef5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.6.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+/* Right outer join / NL */
+
+use test;
+
+select ds1.b, ds2.y
+from ds2 right outer join ds1 on (ds1.a - ds2.x >= 0 and ds2.x - ds1.a >= 0)
+order by ds1.b, ds2.y;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.7.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.7.ddl.sqlpp
new file mode 100644
index 0000000..a58785a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.7.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+create index idx_x on ds2(x:int);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.8.query.sqlpp
new file mode 100644
index 0000000..279ed0f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.8.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+/* Left outer join / Index-NL  */
+
+use test;
+
+select ds1.b, ds2.y
+from ds1 left outer join ds2 on bigint(ds1.a) /* +indexnl */ = ds2.x
+order by ds1.b, ds2.y;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.9.query.sqlpp
new file mode 100644
index 0000000..e6b0c54
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_join_01/outer_join_01.9.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+/* Right outer join / Index-NL  */
+
+use test;
+
+select ds1.b, ds2.y
+from ds2 right outer join ds1 on bigint(ds1.a) /* +indexnl */ = ds2.x
+order by ds1.b, ds2.y;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_unnest_01/outer_unnest_01.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_unnest_01/outer_unnest_01.1.ddl.sqlpp
new file mode 100644
index 0000000..5f5c3fe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_unnest_01/outer_unnest_01.1.ddl.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/* Test that outer unnest emits NULLs if there are no items in the input list */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create dataset ds1(id int) open type primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_unnest_01/outer_unnest_01.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_unnest_01/outer_unnest_01.2.update.sqlpp
new file mode 100644
index 0000000..1fb895a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_unnest_01/outer_unnest_01.2.update.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into ds1 ({"id": 1, "a": [1] });
+insert into ds1 ({"id": 2, "a": [1, 2] });
+insert into ds1 ({"id": 3, "a": [] });
+insert into ds1 ({"id": 4, "a": null });
+insert into ds1 ({"id": 5 /*a:missing*/ });
+insert into ds1 ({"id": 6, "a": [1,2,3] });
+insert into ds1 ({"id": 7, "a": [1, missing, null, 4] });
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_unnest_01/outer_unnest_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_unnest_01/outer_unnest_01.3.query.sqlpp
new file mode 100644
index 0000000..ec5c8af
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_unnest_01/outer_unnest_01.3.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+/* Test that outer unnest emits NULLs if there are no items in the input list */
+
+use test;
+
+select ds1.id, ai
+from ds1 left outer unnest ds1.a as ai
+order by ds1.id, ai;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_unnest_01/outer_unnest_01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_unnest_01/outer_unnest_01.4.query.sqlpp
new file mode 100644
index 0000000..02b34a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/sql-compat/outer_unnest_01/outer_unnest_01.4.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+// requesttype=application/json
+// param sql-compat:json=true
+
+/* Test that outer unnest emits NULLs for positional variable as well */
+
+use test;
+
+select ds1.id, ai, pi
+from ds1 left outer unnest ds1.a as ai at pi
+order by ds1.id, ai;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.3.adm
new file mode 100644
index 0000000..e4704d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.3.adm
@@ -0,0 +1,9 @@
+{ "b": 1, "y": null }
+{ "b": 2, "y": null }
+{ "b": 3, "y": 100 }
+{ "b": 3, "y": 101 }
+{ "b": 4, "y": 100 }
+{ "b": 4, "y": 101 }
+{ "b": 5, "y": 102 }
+{ "b": 6, "y": 102 }
+{ "b": 7, "y": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.4.adm
new file mode 100644
index 0000000..e4704d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.4.adm
@@ -0,0 +1,9 @@
+{ "b": 1, "y": null }
+{ "b": 2, "y": null }
+{ "b": 3, "y": 100 }
+{ "b": 3, "y": 101 }
+{ "b": 4, "y": 100 }
+{ "b": 4, "y": 101 }
+{ "b": 5, "y": 102 }
+{ "b": 6, "y": 102 }
+{ "b": 7, "y": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.5.adm
new file mode 100644
index 0000000..e4704d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.5.adm
@@ -0,0 +1,9 @@
+{ "b": 1, "y": null }
+{ "b": 2, "y": null }
+{ "b": 3, "y": 100 }
+{ "b": 3, "y": 101 }
+{ "b": 4, "y": 100 }
+{ "b": 4, "y": 101 }
+{ "b": 5, "y": 102 }
+{ "b": 6, "y": 102 }
+{ "b": 7, "y": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.6.adm
new file mode 100644
index 0000000..e4704d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.6.adm
@@ -0,0 +1,9 @@
+{ "b": 1, "y": null }
+{ "b": 2, "y": null }
+{ "b": 3, "y": 100 }
+{ "b": 3, "y": 101 }
+{ "b": 4, "y": 100 }
+{ "b": 4, "y": 101 }
+{ "b": 5, "y": 102 }
+{ "b": 6, "y": 102 }
+{ "b": 7, "y": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.8.adm
new file mode 100644
index 0000000..e4704d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.8.adm
@@ -0,0 +1,9 @@
+{ "b": 1, "y": null }
+{ "b": 2, "y": null }
+{ "b": 3, "y": 100 }
+{ "b": 3, "y": 101 }
+{ "b": 4, "y": 100 }
+{ "b": 4, "y": 101 }
+{ "b": 5, "y": 102 }
+{ "b": 6, "y": 102 }
+{ "b": 7, "y": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.9.adm
new file mode 100644
index 0000000..e4704d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_join_01/outer_join_01.9.adm
@@ -0,0 +1,9 @@
+{ "b": 1, "y": null }
+{ "b": 2, "y": null }
+{ "b": 3, "y": 100 }
+{ "b": 3, "y": 101 }
+{ "b": 4, "y": 100 }
+{ "b": 4, "y": 101 }
+{ "b": 5, "y": 102 }
+{ "b": 6, "y": 102 }
+{ "b": 7, "y": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_unnest_01/outer_unnest_01.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_unnest_01/outer_unnest_01.3.adm
new file mode 100644
index 0000000..5e02a06
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_unnest_01/outer_unnest_01.3.adm
@@ -0,0 +1,13 @@
+{ "id": 1, "ai": 1 }
+{ "id": 2, "ai": 1 }
+{ "id": 2, "ai": 2 }
+{ "id": 3, "ai": null }
+{ "id": 4, "ai": null }
+{ "id": 5, "ai": null }
+{ "id": 6, "ai": 1 }
+{ "id": 6, "ai": 2 }
+{ "id": 6, "ai": 3 }
+{ "id": 7 }
+{ "id": 7, "ai": null }
+{ "id": 7, "ai": 1 }
+{ "id": 7, "ai": 4 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_unnest_01/outer_unnest_01.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_unnest_01/outer_unnest_01.4.adm
new file mode 100644
index 0000000..d0a8bd8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/sql-compat/outer_unnest_01/outer_unnest_01.4.adm
@@ -0,0 +1,13 @@
+{ "id": 1, "pi": 1, "ai": 1 }
+{ "id": 2, "pi": 1, "ai": 1 }
+{ "id": 2, "pi": 2, "ai": 2 }
+{ "id": 3, "pi": null, "ai": null }
+{ "id": 4, "pi": null, "ai": null }
+{ "id": 5, "pi": null, "ai": null }
+{ "id": 6, "pi": 1, "ai": 1 }
+{ "id": 6, "pi": 2, "ai": 2 }
+{ "id": 6, "pi": 3, "ai": 3 }
+{ "id": 7, "pi": 2 }
+{ "id": 7, "pi": 3, "ai": null }
+{ "id": 7, "pi": 1, "ai": 1 }
+{ "id": 7, "pi": 4, "ai": 4 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index cc0f92a..4cafa0e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -10164,6 +10164,16 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="sql-compat">
+      <compilation-unit name="outer_join_01">
+        <output-dir compare="Text">outer_join_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="sql-compat">
+      <compilation-unit name="outer_unnest_01">
+        <output-dir compare="Text">outer_unnest_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="sql-compat">
       <compilation-unit name="select_star_01">
         <output-dir compare="Text">select_star_01</output-dir>
       </compilation-unit>
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorDescriptor.java
index 5ecd283..56bedea 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorDescriptor.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorDescriptor.java
@@ -40,7 +40,7 @@
             int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, int version) {
         super(spec, outRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory,
                 retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, minFilterFieldIndexes,
-                maxFilterFieldIndexes, false);
+                maxFilterFieldIndexes, false, null);
         this.version = version;
     }
 
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorNodePushable.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorNodePushable.java
index 80ebb4c..4e669c3 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorNodePushable.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorNodePushable.java
@@ -46,7 +46,8 @@
             boolean retainMissing, IMissingWriterFactory missingWriterFactory,
             ISearchOperationCallbackFactory searchCallbackFactory, int version) throws HyracksDataException {
         super(ctx, partition, intputRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive, null, null,
-                indexHelperFactory, retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, false);
+                indexHelperFactory, retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, false,
+                null);
         this.version = version;
     }
 
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorDescriptor.java
index 493d3ba..bd6c922 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorDescriptor.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorDescriptor.java
@@ -41,7 +41,7 @@
             int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, int version) {
         super(spec, outRecDesc, keyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput,
                 retainMissing, missingWriterFactory, searchCallbackFactory, minFilterFieldIndexes,
-                maxFilterFieldIndexes, false);
+                maxFilterFieldIndexes, false, null);
         this.version = version;
     }
 
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorNodePushable.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorNodePushable.java
index 0b294de..508cff7 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorNodePushable.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorNodePushable.java
@@ -46,7 +46,7 @@
             IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
             int version) throws HyracksDataException {
         super(ctx, partition, inputRecDesc, keyFields, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
-                retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, false);
+                retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, false, null);
         this.version = version;
     }
 
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/JoinClause.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/JoinClause.java
index 18b7a24..be3ebda 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/JoinClause.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/JoinClause.java
@@ -19,8 +19,11 @@
 
 package org.apache.asterix.lang.sqlpp.clause;
 
+import java.util.Objects;
+
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.base.Literal;
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.asterix.lang.sqlpp.optype.JoinType;
@@ -30,10 +33,21 @@
 
     private final JoinType joinType;
 
+    private Literal.Type outerJoinMissingValueType;
+
     public JoinClause(JoinType joinType, Expression rightExpr, VariableExpr rightVar, VariableExpr rightPosVar,
-            Expression conditionExpr) {
+            Expression conditionExpr, Literal.Type outerJoinMissingValueType) {
         super(rightExpr, rightVar, rightPosVar, conditionExpr);
         this.joinType = joinType;
+        setOuterJoinMissingValueType(outerJoinMissingValueType);
+    }
+
+    public Literal.Type getOuterJoinMissingValueType() {
+        return outerJoinMissingValueType;
+    }
+
+    public void setOuterJoinMissingValueType(Literal.Type outerJoinMissingValueType) {
+        this.outerJoinMissingValueType = validateMissingValueType(joinType, outerJoinMissingValueType);
     }
 
     @Override
@@ -52,7 +66,8 @@
 
     @Override
     public int hashCode() {
-        return 31 * super.hashCode() + joinType.hashCode();
+        return 31 * super.hashCode() + 31 * joinType.hashCode()
+                + (outerJoinMissingValueType != null ? outerJoinMissingValueType.hashCode() : 0);
     }
 
     @Override
@@ -64,6 +79,28 @@
             return false;
         }
         JoinClause target = (JoinClause) object;
-        return super.equals(target) && joinType.equals(target.getJoinType());
+        return super.equals(target) && joinType.equals(target.getJoinType())
+                && Objects.equals(outerJoinMissingValueType, target.outerJoinMissingValueType);
+    }
+
+    private static Literal.Type validateMissingValueType(JoinType joinType, Literal.Type missingValueType) {
+        switch (joinType) {
+            case INNER:
+                if (missingValueType != null) {
+                    throw new IllegalArgumentException(String.valueOf(missingValueType));
+                }
+                return null;
+            case LEFTOUTER:
+            case RIGHTOUTER:
+                switch (Objects.requireNonNull(missingValueType)) {
+                    case MISSING:
+                    case NULL:
+                        return missingValueType;
+                    default:
+                        throw new IllegalArgumentException(String.valueOf(missingValueType));
+                }
+            default:
+                throw new IllegalStateException(String.valueOf(joinType));
+        }
     }
 }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/UnnestClause.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/UnnestClause.java
index f035be4..054ff26 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/UnnestClause.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/clause/UnnestClause.java
@@ -19,8 +19,11 @@
 
 package org.apache.asterix.lang.sqlpp.clause;
 
+import java.util.Objects;
+
 import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.base.Literal;
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.asterix.lang.sqlpp.optype.UnnestType;
@@ -30,9 +33,21 @@
 
     private final UnnestType unnestType;
 
-    public UnnestClause(UnnestType unnestType, Expression rightExpr, VariableExpr rightVar, VariableExpr rightPosVar) {
+    private Literal.Type outerUnnestMissingValueType;
+
+    public UnnestClause(UnnestType unnestType, Expression rightExpr, VariableExpr rightVar, VariableExpr rightPosVar,
+            Literal.Type outerUnnestMissingValueType) {
         super(rightExpr, rightVar, rightPosVar);
         this.unnestType = unnestType;
+        setOuterUnnestMissingValueType(outerUnnestMissingValueType);
+    }
+
+    public Literal.Type getOuterUnnestMissingValueType() {
+        return outerUnnestMissingValueType;
+    }
+
+    public void setOuterUnnestMissingValueType(Literal.Type outerUnnestMissingValueType) {
+        this.outerUnnestMissingValueType = validateMissingValueType(unnestType, outerUnnestMissingValueType);
     }
 
     @Override
@@ -51,7 +66,8 @@
 
     @Override
     public int hashCode() {
-        return 31 * super.hashCode() + unnestType.hashCode();
+        return 31 * super.hashCode() + 31 * unnestType.hashCode()
+                + +(outerUnnestMissingValueType != null ? outerUnnestMissingValueType.hashCode() : 0);
     }
 
     @Override
@@ -63,6 +79,27 @@
             return false;
         }
         UnnestClause target = (UnnestClause) object;
-        return super.equals(target) && unnestType.equals(target.getUnnestType());
+        return super.equals(target) && unnestType.equals(target.getUnnestType())
+                && Objects.equals(outerUnnestMissingValueType, target.outerUnnestMissingValueType);
+    }
+
+    private static Literal.Type validateMissingValueType(UnnestType unnestType, Literal.Type missingValueType) {
+        switch (unnestType) {
+            case INNER:
+                if (missingValueType != null) {
+                    throw new IllegalArgumentException(String.valueOf(missingValueType));
+                }
+                return null;
+            case LEFTOUTER:
+                switch (Objects.requireNonNull(missingValueType)) {
+                    case MISSING:
+                    case NULL:
+                        return missingValueType;
+                    default:
+                        throw new IllegalArgumentException(String.valueOf(missingValueType));
+                }
+            default:
+                throw new IllegalStateException(String.valueOf(unnestType));
+        }
     }
 }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlCompatRewriteVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlCompatRewriteVisitor.java
index 1b44377..a0939c3 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlCompatRewriteVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlCompatRewriteVisitor.java
@@ -28,6 +28,7 @@
 import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.ILangExpression;
+import org.apache.asterix.lang.common.base.Literal;
 import org.apache.asterix.lang.common.clause.LetClause;
 import org.apache.asterix.lang.common.clause.LimitClause;
 import org.apache.asterix.lang.common.expression.CallExpr;
@@ -114,6 +115,17 @@
         if (expr.getKind() == Expression.Kind.SELECT_EXPRESSION) {
             annotateSubqueryNoCoercion((SelectExpression) expr);
         }
+        switch (joinClause.getJoinType()) {
+            case LEFTOUTER:
+            case RIGHTOUTER:
+                joinClause.setOuterJoinMissingValueType(Literal.Type.NULL);
+                break;
+            case INNER:
+                break;
+            default:
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, joinClause.getSourceLocation(),
+                        String.valueOf(joinClause.getJoinType()));
+        }
         return super.visit(joinClause, arg);
     }
 
@@ -123,6 +135,17 @@
         if (expr.getKind() == Expression.Kind.SELECT_EXPRESSION) {
             annotateSubqueryNoCoercion((SelectExpression) expr);
         }
+        // keep UNNEST clause aligned with JOIN clause when it comes to producing NULL values
+        switch (unnestClause.getUnnestType()) {
+            case LEFTOUTER:
+                unnestClause.setOuterUnnestMissingValueType(Literal.Type.NULL);
+                break;
+            case INNER:
+                break;
+            default:
+                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, unnestClause.getSourceLocation(),
+                        String.valueOf(unnestClause.getUnnestType()));
+        }
         return super.visit(unnestClause, arg);
     }
 
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppRightJoinRewriteVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppRightJoinRewriteVisitor.java
index 8a3b913..26c6915 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppRightJoinRewriteVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppRightJoinRewriteVisitor.java
@@ -32,6 +32,7 @@
 import org.apache.asterix.lang.common.base.Clause;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.ILangExpression;
+import org.apache.asterix.lang.common.base.Literal;
 import org.apache.asterix.lang.common.clause.LetClause;
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
@@ -161,9 +162,10 @@
                 VariableExpr rightVar = joinClause.getRightVariable();
                 VariableExpr rightPosVar = joinClause.getPositionalVariable();
                 Expression condExpr = joinClause.getConditionExpression();
+                Literal.Type outerMissingValueType = joinClause.getOuterJoinMissingValueType();
                 if (i == 0) {
-                    JoinClause newJoinClause =
-                            new JoinClause(JoinType.LEFTOUTER, fromExpr, fromVar, fromPosVar, condExpr);
+                    JoinClause newJoinClause = new JoinClause(JoinType.LEFTOUTER, fromExpr, fromVar, fromPosVar,
+                            condExpr, outerMissingValueType);
                     newJoinClause.setSourceLocation(joinClauseSourceLoc);
 
                     fromExpr = rightExpr;
@@ -249,7 +251,7 @@
                                                 ? newVariableExpr(newPrecedingClauseRightPosVar,
                                                         precedingClauseRightPosVarSourceLoc)
                                                 : null,
-                                        newCondExpr);
+                                        newCondExpr, joinPrecedingClause.getOuterJoinMissingValueType());
                                 newPrecedingClause.setSourceLocation(precedingClauseSourceLoc);
                                 break;
                             case UNNEST_CLAUSE:
@@ -260,10 +262,10 @@
                                 newPrecedingClause = new UnnestClause(unnestPrecedingClause.getUnnestType(),
                                         newRightExpr,
                                         newVariableExpr(newPrecedingClauseRightVar, precedingClauseRightVarSourceLoc),
-                                        newPrecedingClauseRightPosVar != null
-                                                ? newVariableExpr(newPrecedingClauseRightPosVar,
-                                                        precedingClauseRightPosVarSourceLoc)
-                                                : null);
+                                        newPrecedingClauseRightPosVar != null ? newVariableExpr(
+                                                newPrecedingClauseRightPosVar, precedingClauseRightPosVarSourceLoc)
+                                                : null,
+                                        unnestPrecedingClause.getOuterUnnestMissingValueType());
                                 newPrecedingClause.setSourceLocation(precedingClauseSourceLoc);
                                 break;
                             default:
@@ -291,8 +293,8 @@
                     Expression newCondExpr = SqlppRewriteUtil.substituteExpression(
                             (Expression) SqlppRewriteUtil.deepCopy(condExpr), substMapOuterFinal, context);
 
-                    JoinClause newJoinClause =
-                            new JoinClause(JoinType.LEFTOUTER, newRightExpr, newRightVarExpr, null, newCondExpr);
+                    JoinClause newJoinClause = new JoinClause(JoinType.LEFTOUTER, newRightExpr, newRightVarExpr, null,
+                            newCondExpr, outerMissingValueType);
                     newJoinClause.setSourceLocation(joinClauseSourceLoc);
 
                     fromExpr = rightExpr;
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/DeepCopyVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/DeepCopyVisitor.java
index 283b416..d74d047 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/DeepCopyVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/DeepCopyVisitor.java
@@ -112,7 +112,7 @@
                 : (VariableExpr) joinClause.getPositionalVariable().accept(this, arg);
         Expression conditionExpresion = (Expression) joinClause.getConditionExpression().accept(this, arg);
         JoinClause copy = new JoinClause(joinClause.getJoinType(), rightExpression, rightVar, rightPositionVar,
-                conditionExpresion);
+                conditionExpresion, joinClause.getOuterJoinMissingValueType());
         copy.setSourceLocation(joinClause.getSourceLocation());
         return copy;
     }
@@ -136,7 +136,8 @@
         VariableExpr rightVar = (VariableExpr) unnestClause.getRightVariable().accept(this, arg);
         VariableExpr rightPositionVar = unnestClause.getPositionalVariable() == null ? null
                 : (VariableExpr) unnestClause.getPositionalVariable().accept(this, arg);
-        UnnestClause copy = new UnnestClause(unnestClause.getUnnestType(), rightExpression, rightVar, rightPositionVar);
+        UnnestClause copy = new UnnestClause(unnestClause.getUnnestType(), rightExpression, rightVar, rightPositionVar,
+                unnestClause.getOuterUnnestMissingValueType());
         copy.setSourceLocation(unnestClause.getSourceLocation());
         return copy;
     }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppAstPrintVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppAstPrintVisitor.java
index db663cd..c803d83 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppAstPrintVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppAstPrintVisitor.java
@@ -28,6 +28,7 @@
 import org.apache.asterix.common.metadata.DatasetFullyQualifiedName;
 import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.base.Literal;
 import org.apache.asterix.lang.common.clause.GroupbyClause;
 import org.apache.asterix.lang.common.clause.LetClause;
 import org.apache.asterix.lang.common.clause.OrderbyClause;
@@ -109,6 +110,7 @@
             out.print(" AT ");
             joinClause.getPositionalVariable().accept(this, 0);
         }
+        out.print(Literal.Type.NULL.equals(joinClause.getOuterJoinMissingValueType()) ? "(OR NULL) " : "");
         out.println(skip(step + 1) + "ON");
         joinClause.getConditionExpression().accept(this, step + 1);
         return null;
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppCloneAndSubstituteVariablesVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppCloneAndSubstituteVariablesVisitor.java
index e664d49..ce0d0a1 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppCloneAndSubstituteVariablesVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/SqlppCloneAndSubstituteVariablesVisitor.java
@@ -143,8 +143,8 @@
         // The condition can refer to the newRightVar and newRightPosVar.
         Expression conditionExpr = (Expression) joinClause.getConditionExpression().accept(this, currentEnv).first;
 
-        JoinClause newJoinClause =
-                new JoinClause(joinClause.getJoinType(), newRightExpr, newRightVar, newRightPosVar, conditionExpr);
+        JoinClause newJoinClause = new JoinClause(joinClause.getJoinType(), newRightExpr, newRightVar, newRightPosVar,
+                conditionExpr, joinClause.getOuterJoinMissingValueType());
         newJoinClause.setSourceLocation(joinClause.getSourceLocation());
         return new Pair<>(newJoinClause, currentEnv);
     }
@@ -193,8 +193,8 @@
             currentEnv.removeSubstitution(newRightPosVar);
         }
         // The condition can refer to the newRightVar and newRightPosVar.
-        UnnestClause newUnnestClause =
-                new UnnestClause(unnestClause.getUnnestType(), rightExpr, newRightVar, newRightPosVar);
+        UnnestClause newUnnestClause = new UnnestClause(unnestClause.getUnnestType(), rightExpr, newRightVar,
+                newRightPosVar, unnestClause.getOuterUnnestMissingValueType());
         newUnnestClause.setSourceLocation(unnestClause.getSourceLocation());
         return new Pair<>(newUnnestClause, currentEnv);
     }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
index 395f63d..4e64944 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
+++ b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
@@ -4841,7 +4841,7 @@
   <JOIN> { startToken = token; } rightInput = JoinClauseRightInput() <ON> conditionExpr = Expression()
   {
     JoinClause joinClause = new JoinClause(joinType, rightInput.first, rightInput.second, rightInput.third,
-      conditionExpr);
+      conditionExpr, joinType == JoinType.INNER ? null : Literal.Type.MISSING);
     return addSourceLocation(joinClause, startToken);
   }
 }
@@ -4856,7 +4856,7 @@
   <JOIN> { startToken = token; } rightInput = JoinClauseRightInput()
   {
     JoinClause joinClause = new JoinClause(JoinType.INNER, rightInput.first, rightInput.second, rightInput.third,
-      new LiteralExpr(TrueLiteral.INSTANCE));
+      new LiteralExpr(TrueLiteral.INSTANCE), null);
     return addSourceLocation(joinClause, startToken);
   }
 }
@@ -4890,7 +4890,8 @@
     if (rightVar == null) {
       rightVar = ExpressionToVariableUtil.getGeneratedVariable(rightExpr, true);
     }
-    UnnestClause unnestClause = new UnnestClause(unnestType, rightExpr, rightVar, posVar);
+    UnnestClause unnestClause = new UnnestClause(unnestType, rightExpr, rightVar, posVar,
+      unnestType == UnnestType.INNER ? null : Literal.Type.MISSING);
     return addSourceLocation(unnestClause, startToken);
   }
 }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
index 06f4f73..1d82b19 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
@@ -132,10 +132,10 @@
 
                 int[] minFilterFieldIndexes = createFilterIndexes(minFilterVars, opSchema);
                 int[] maxFilterFieldIndexes = createFilterIndexes(maxFilterVars, opSchema);
-                return metadataProvider.buildBtreeRuntime(jobSpec, opSchema, typeEnv, context, true, false,
+                return metadataProvider.buildBtreeRuntime(jobSpec, opSchema, typeEnv, context, true, false, null,
                         ((DatasetDataSource) dataSource).getDataset(), primaryIndex.getIndexName(), null, null, true,
-                        true, false, minFilterFieldIndexes, maxFilterFieldIndexes, tupleFilterFactory, outputLimit,
-                        false, false);
+                        true, false, null, minFilterFieldIndexes, maxFilterFieldIndexes, tupleFilterFactory,
+                        outputLimit, false, false);
             default:
                 throw new AlgebricksException("Unknown datasource type");
         }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
index 011d862..5ce97eb 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
@@ -540,8 +540,9 @@
 
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildBtreeRuntime(JobSpecification jobSpec,
             IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv, JobGenContext context, boolean retainInput,
-            boolean retainMissing, Dataset dataset, String indexName, int[] lowKeyFields, int[] highKeyFields,
-            boolean lowKeyInclusive, boolean highKeyInclusive, boolean propagateFilter, int[] minFilterFieldIndexes,
+            boolean retainMissing, IMissingWriterFactory nonMatchWriterFactory, Dataset dataset, String indexName,
+            int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
+            boolean propagateFilter, IMissingWriterFactory nonFilterWriterFactory, int[] minFilterFieldIndexes,
             int[] maxFilterFieldIndexes, ITupleFilterFactory tupleFilterFactory, long outputLimit,
             boolean isIndexOnlyPlan, boolean isPrimaryIndexPointSearch) throws AlgebricksException {
         boolean isSecondary = true;
@@ -601,27 +602,28 @@
             btreeSearchOp = !isSecondary && isPrimaryIndexPointSearch
                     ? new LSMBTreeBatchPointSearchOperatorDescriptor(jobSpec, outputRecDesc, lowKeyFields,
                             highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput,
-                            retainMissing, context.getMissingWriterFactory(), searchCallbackFactory,
-                            minFilterFieldIndexes, maxFilterFieldIndexes, tupleFilterFactory, outputLimit)
+                            retainMissing, nonMatchWriterFactory, searchCallbackFactory, minFilterFieldIndexes,
+                            maxFilterFieldIndexes, tupleFilterFactory, outputLimit)
                     : new BTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, lowKeyFields, highKeyFields,
                             lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput, retainMissing,
-                            context.getMissingWriterFactory(), searchCallbackFactory, minFilterFieldIndexes,
-                            maxFilterFieldIndexes, propagateFilter, tupleFilterFactory, outputLimit,
+                            nonMatchWriterFactory, searchCallbackFactory, minFilterFieldIndexes, maxFilterFieldIndexes,
+                            propagateFilter, nonFilterWriterFactory, tupleFilterFactory, outputLimit,
                             proceedIndexOnlyPlan, failValueForIndexOnlyPlan, successValueForIndexOnlyPlan);
         } else {
             btreeSearchOp = new ExternalBTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, lowKeyFields,
                     highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput, retainMissing,
-                    context.getMissingWriterFactory(), searchCallbackFactory, minFilterFieldIndexes,
-                    maxFilterFieldIndexes, ExternalDatasetsRegistry.INSTANCE.getAndLockDatasetVersion(dataset, this));
+                    nonMatchWriterFactory, searchCallbackFactory, minFilterFieldIndexes, maxFilterFieldIndexes,
+                    ExternalDatasetsRegistry.INSTANCE.getAndLockDatasetVersion(dataset, this));
         }
         return new Pair<>(btreeSearchOp, spPc.second);
     }
 
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildRtreeRuntime(JobSpecification jobSpec,
             List<LogicalVariable> outputVars, IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv,
-            JobGenContext context, boolean retainInput, boolean retainMissing, Dataset dataset, String indexName,
-            int[] keyFields, boolean propagateFilter, int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes,
-            boolean isIndexOnlyPlan) throws AlgebricksException {
+            JobGenContext context, boolean retainInput, boolean retainMissing,
+            IMissingWriterFactory nonMatchWriterFactory, Dataset dataset, String indexName, int[] keyFields,
+            boolean propagateFilter, IMissingWriterFactory nonFilterWriterFactory, int[] minFilterFieldIndexes,
+            int[] maxFilterFieldIndexes, boolean isIndexOnlyPlan) throws AlgebricksException {
         int numPrimaryKeys = dataset.getPrimaryKeys().size();
         Index secondaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
                 dataset.getDatasetName(), indexName);
@@ -667,13 +669,13 @@
                 new IndexDataflowHelperFactory(storageComponentProvider.getStorageManager(), spPc.first);
         if (dataset.getDatasetType() == DatasetType.INTERNAL) {
             rtreeSearchOp = new RTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, keyFields, true, true,
-                    indexDataflowHelperFactory, retainInput, retainMissing, context.getMissingWriterFactory(),
+                    indexDataflowHelperFactory, retainInput, retainMissing, nonMatchWriterFactory,
                     searchCallbackFactory, minFilterFieldIndexes, maxFilterFieldIndexes, propagateFilter,
-                    isIndexOnlyPlan, failValueForIndexOnlyPlan, successValueForIndexOnlyPlan);
+                    nonFilterWriterFactory, isIndexOnlyPlan, failValueForIndexOnlyPlan, successValueForIndexOnlyPlan);
         } else {
             // Create the operator
             rtreeSearchOp = new ExternalRTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, keyFields, true, true,
-                    indexDataflowHelperFactory, retainInput, retainMissing, context.getMissingWriterFactory(),
+                    indexDataflowHelperFactory, retainInput, retainMissing, nonMatchWriterFactory,
                     searchCallbackFactory, minFilterFieldIndexes, maxFilterFieldIndexes,
                     ExternalDatasetsRegistry.INSTANCE.getAndLockDatasetVersion(dataset, this));
         }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
index 143dc4c..6529794 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
@@ -19,7 +19,6 @@
 package org.apache.asterix.metadata.utils;
 
 import static org.apache.asterix.common.utils.IdentifierUtil.dataset;
-import static org.apache.asterix.common.utils.IdentifierUtil.dataverse;
 
 import java.io.DataOutput;
 import java.util.ArrayList;
@@ -417,7 +416,7 @@
                 metadataProvider.getStorageComponentProvider().getStorageManager(), primaryFileSplitProvider);
         BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec,
                 dataset.getPrimaryRecordDescriptor(metadataProvider), lowKeyFields, highKeyFields, true, true,
-                indexHelperFactory, false, false, null, searchCallbackFactory, null, null, false);
+                indexHelperFactory, false, false, null, searchCallbackFactory, null, null, false, null);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, primarySearchOp,
                 primaryPartitionConstraint);
         return primarySearchOp;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/MissableTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/MissableTypeComputer.java
index 91b5d19..b51ebb4 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/MissableTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/MissableTypeComputer.java
@@ -19,10 +19,11 @@
 package org.apache.asterix.dataflow.data.common;
 
 import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
+import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.types.TypeHelper;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IMissableTypeComputer;
 
 public class MissableTypeComputer implements IMissableTypeComputer {
@@ -33,9 +34,10 @@
     }
 
     @Override
-    public IAType makeMissableType(Object type) throws AlgebricksException {
+    public IAType makeMissableType(Object type) {
         IAType t = (IAType) type;
-        return AUnionType.createMissableType(t);
+        return TypeHelper.canBeMissing(t) ? t
+                : t.getTypeTag() == ATypeTag.NULL ? BuiltinType.ANY : AUnionType.createMissableType(t);
     }
 
     @Override
@@ -45,8 +47,45 @@
     }
 
     @Override
+    public Object makeNullableType(Object type) {
+        IAType t = (IAType) type;
+        return TypeHelper.canBeNull(t) ? t
+                : t.getTypeTag() == ATypeTag.MISSING ? BuiltinType.ANY : AUnionType.createNullableType(t);
+    }
+
+    @Override
+    public boolean canBeNull(Object type) {
+        IAType t = (IAType) type;
+        return TypeHelper.canBeNull(t);
+    }
+
+    @Override
     public Object getNonOptionalType(Object type) {
         IAType t = (IAType) type;
         return TypeComputeUtils.getActualType(t);
     }
+
+    @Override
+    public Object getNonMissableType(Object type) {
+        IAType t = (IAType) type;
+        if (t.getTypeTag() == ATypeTag.UNION) {
+            AUnionType ut = ((AUnionType) t);
+            IAType primeType = ut.getActualType();
+            return ut.isNullableType() ? AUnionType.createNullableType(primeType) : primeType;
+        } else {
+            return t;
+        }
+    }
+
+    @Override
+    public Object getNonNullableType(Object type) {
+        IAType t = (IAType) type;
+        if (t.getTypeTag() == ATypeTag.UNION) {
+            AUnionType ut = ((AUnionType) t);
+            IAType primeType = ut.getActualType();
+            return ut.isMissableType() ? AUnionType.createMissableType(primeType) : primeType;
+        } else {
+            return t;
+        }
+    }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/NullWriterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/NullWriterFactory.java
new file mode 100644
index 0000000..0a5ef58
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/NullWriterFactory.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.dataflow.data.nontagged;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.api.dataflow.value.IMissingWriter;
+import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public final class NullWriterFactory implements IMissingWriterFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    public static final NullWriterFactory INSTANCE = new NullWriterFactory();
+
+    private NullWriterFactory() {
+    }
+
+    @Override
+    public IMissingWriter createMissingWriter() {
+        return NullWriterFactory::writeNull;
+    }
+
+    private static void writeNull(DataOutput out) throws HyracksDataException {
+        try {
+            out.writeByte(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+        } catch (IOException e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/base/IDataFormat.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/base/IDataFormat.java
index 27b7802..82af52e 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/base/IDataFormat.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/base/IDataFormat.java
@@ -69,6 +69,8 @@
 
     public IMissingWriterFactory getMissingWriterFactory();
 
+    public IMissingWriterFactory getNullWriterFactory();
+
     public IUnnestingPositionWriterFactory getUnnestingPositionWriterFactory();
 
     public Triple<IScalarEvaluatorFactory, ScalarFunctionCallExpression, IAType> partitioningEvaluatorFactory(
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/constants/AsterixConstantValue.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/constants/AsterixConstantValue.java
index 337e966..8b5da27 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/constants/AsterixConstantValue.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/constants/AsterixConstantValue.java
@@ -22,6 +22,7 @@
 import org.apache.asterix.om.base.AMissing;
 import org.apache.asterix.om.base.ANull;
 import org.apache.asterix.om.base.IAObject;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
 
 public class AsterixConstantValue implements IAlgebricksConstantValue {
@@ -63,15 +64,27 @@
 
     @Override
     public boolean equals(Object o) {
-        if (!(o instanceof AsterixConstantValue)) {
+        if (o instanceof AsterixConstantValue) {
+            AsterixConstantValue v2 = (AsterixConstantValue) o;
+            return object.deepEqual(v2.getObject());
+        } else if (o instanceof IAlgebricksConstantValue) {
+            return o.equals(this);
+        } else {
             return false;
         }
-        AsterixConstantValue v2 = (AsterixConstantValue) o;
-        return object.deepEqual(v2.getObject());
     }
 
     @Override
     public int hashCode() {
-        return object.hash();
+        switch (object.getType().getTypeTag()) {
+            case MISSING:
+                return ConstantExpression.MISSING.hashCode();
+            case NULL:
+                return ConstantExpression.NULL.hashCode();
+            case BOOLEAN:
+                return (isTrue() ? ConstantExpression.TRUE : ConstantExpression.FALSE).hashCode();
+            default:
+                return object.hash();
+        }
     }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
index 0a1a8e9..0c9f0dd 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
@@ -1047,6 +1047,8 @@
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "percent-rank-impl", FunctionIdentifier.VARARGS);
     public static final FunctionIdentifier WIN_MARK_FIRST_MISSING_IMPL = new FunctionIdentifier(
             FunctionConstants.ASTERIX_NS, "win-mark-first-missing-impl", FunctionIdentifier.VARARGS);
+    public static final FunctionIdentifier WIN_MARK_FIRST_NULL_IMPL = new FunctionIdentifier(
+            FunctionConstants.ASTERIX_NS, "win-mark-first-null-impl", FunctionIdentifier.VARARGS);
     public static final FunctionIdentifier WIN_PARTITION_LENGTH_IMPL =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "win-partition-length-impl", 0);
 
@@ -2258,6 +2260,7 @@
         addFunction(PERCENT_RANK, ADoubleTypeComputer.INSTANCE, false);
         addFunction(PERCENT_RANK_IMPL, ADoubleTypeComputer.INSTANCE, false);
         addPrivateFunction(WIN_MARK_FIRST_MISSING_IMPL, ABooleanTypeComputer.INSTANCE, false);
+        addPrivateFunction(WIN_MARK_FIRST_NULL_IMPL, ABooleanTypeComputer.INSTANCE, false);
         addPrivateFunction(WIN_PARTITION_LENGTH_IMPL, AInt64TypeComputer.INSTANCE, false);
 
         // Similarity functions
@@ -3230,6 +3233,7 @@
         addWindowFunction(RATIO_TO_REPORT, RATIO_TO_REPORT_IMPL, HAS_LIST_ARG);
         addWindowFunction(ROW_NUMBER, ROW_NUMBER_IMPL, NO_FRAME_CLAUSE);
         addWindowFunction(null, WIN_MARK_FIRST_MISSING_IMPL, NO_FRAME_CLAUSE, INJECT_ORDER_ARGS);
+        addWindowFunction(null, WIN_MARK_FIRST_NULL_IMPL, NO_FRAME_CLAUSE, INJECT_ORDER_ARGS);
         addWindowFunction(null, WIN_PARTITION_LENGTH_IMPL, NO_FRAME_CLAUSE, MATERIALIZE_PARTITION);
     }
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/TypeHelper.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/TypeHelper.java
index b444053..645c12a 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/TypeHelper.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/TypeHelper.java
@@ -29,6 +29,7 @@
     public static boolean canBeMissing(IAType t) {
         switch (t.getTypeTag()) {
             case MISSING:
+            case ANY:
                 return true;
             case UNION:
                 return ((AUnionType) t).isMissableType();
@@ -37,6 +38,18 @@
         }
     }
 
+    public static boolean canBeNull(IAType t) {
+        switch (t.getTypeTag()) {
+            case NULL:
+            case ANY:
+                return true;
+            case UNION:
+                return ((AUnionType) t).isNullableType();
+            default:
+                return false;
+        }
+    }
+
     public static boolean canBeUnknown(IAType t) {
         switch (t.getTypeTag()) {
             case MISSING:
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java
index f2685f4..8c8b7e6 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java
@@ -24,6 +24,7 @@
 
 import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.dataflow.data.nontagged.MissingWriterFactory;
+import org.apache.asterix.dataflow.data.nontagged.NullWriterFactory;
 import org.apache.asterix.formats.base.IDataFormat;
 import org.apache.asterix.formats.nontagged.ADMPrinterFactoryProvider;
 import org.apache.asterix.formats.nontagged.BinaryBooleanInspector;
@@ -352,6 +353,11 @@
     }
 
     @Override
+    public IMissingWriterFactory getNullWriterFactory() {
+        return NullWriterFactory.INSTANCE;
+    }
+
+    @Override
     public IUnnestingPositionWriterFactory getUnnestingPositionWriterFactory() {
         return UnnestingPositionWriterFactory.INSTANCE;
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
index f2d1f7a..35993cf 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
@@ -619,6 +619,7 @@
 import org.apache.asterix.runtime.runningaggregates.std.RowNumberRunningAggregateDescriptor;
 import org.apache.asterix.runtime.runningaggregates.std.TidRunningAggregateDescriptor;
 import org.apache.asterix.runtime.runningaggregates.std.WinMarkFirstMissingRunningAggregateDescriptor;
+import org.apache.asterix.runtime.runningaggregates.std.WinMarkFirstNullRunningAggregateDescriptor;
 import org.apache.asterix.runtime.runningaggregates.std.WinPartitionLenRunningAggregateDescriptor;
 import org.apache.asterix.runtime.unnestingfunctions.std.RangeDescriptor;
 import org.apache.asterix.runtime.unnestingfunctions.std.ScanCollectionDescriptor;
@@ -909,6 +910,7 @@
         fc.add(RowNumberRunningAggregateDescriptor.FACTORY);
         fc.add(PercentRankRunningAggregateDescriptor.FACTORY);
         fc.add(WinMarkFirstMissingRunningAggregateDescriptor.FACTORY);
+        fc.add(WinMarkFirstNullRunningAggregateDescriptor.FACTORY);
         fc.add(WinPartitionLenRunningAggregateDescriptor.FACTORY);
 
         // boolean functions
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinMarkFirstMissingRunningAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinMarkFirstMissingRunningAggregateDescriptor.java
index 3f5f9f6..8a872c5 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinMarkFirstMissingRunningAggregateDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinMarkFirstMissingRunningAggregateDescriptor.java
@@ -21,6 +21,7 @@
 
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.runtime.runningaggregates.base.AbstractRunningAggregateFunctionDynamicDescriptor;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
@@ -60,7 +61,7 @@
                 for (int i = 0; i < args.length; i++) {
                     evals[i] = args[i].createScalarEvaluator(ctx);
                 }
-                return new WinMarkFirstMissingRunningAggregateEvaluator(evals);
+                return new WinMarkFirstUnknownRunningAggregateEvaluator(ATypeTag.MISSING, evals);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinMarkFirstNullRunningAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinMarkFirstNullRunningAggregateDescriptor.java
new file mode 100644
index 0000000..f8c2f3f
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinMarkFirstNullRunningAggregateDescriptor.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.runningaggregates.std;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.runningaggregates.base.AbstractRunningAggregateFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IRunningAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IRunningAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * This internal window function returns {@code TRUE} in the following two cases:
+ * <ol>
+ * <li>the argument is not NULL</li>
+ * <li>the argument is NULL and it comes from the first tuple in the current window partition</li>
+ * </ol>
+ * In all other cases the function returns {@code FALSE}.
+ * <p>
+ * The underlying assumption is that tuples in each window partition are sorted on the function's argument in the
+ * descending order.
+ */
+public class WinMarkFirstNullRunningAggregateDescriptor extends AbstractRunningAggregateFunctionDynamicDescriptor {
+
+    public static final IFunctionDescriptorFactory FACTORY = WinMarkFirstNullRunningAggregateDescriptor::new;
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public IRunningAggregateEvaluatorFactory createRunningAggregateEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+        return new IRunningAggregateEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IRunningAggregateEvaluator createRunningAggregateEvaluator(IEvaluatorContext ctx)
+                    throws HyracksDataException {
+                IScalarEvaluator[] evals = new IScalarEvaluator[args.length];
+                for (int i = 0; i < args.length; i++) {
+                    evals[i] = args[i].createScalarEvaluator(ctx);
+                }
+                return new WinMarkFirstUnknownRunningAggregateEvaluator(ATypeTag.NULL, evals);
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.WIN_MARK_FIRST_NULL_IMPL;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinMarkFirstMissingRunningAggregateEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinMarkFirstUnknownRunningAggregateEvaluator.java
similarity index 84%
rename from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinMarkFirstMissingRunningAggregateEvaluator.java
rename to asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinMarkFirstUnknownRunningAggregateEvaluator.java
index c872921..93ba4ef 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinMarkFirstMissingRunningAggregateEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/runningaggregates/std/WinMarkFirstUnknownRunningAggregateEvaluator.java
@@ -34,7 +34,7 @@
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-public final class WinMarkFirstMissingRunningAggregateEvaluator implements IWindowAggregateEvaluator {
+public final class WinMarkFirstUnknownRunningAggregateEvaluator implements IWindowAggregateEvaluator {
 
     @SuppressWarnings({ "rawtypes" })
     private final ISerializerDeserializer boolSerde =
@@ -42,6 +42,8 @@
 
     private final IScalarEvaluator[] argEvals;
 
+    private final byte unknownTypeTag;
+
     private final TaggedValuePointable argValue;
 
     private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -50,10 +52,11 @@
 
     private boolean first;
 
-    private boolean firstAllMissing;
+    private boolean firstAllUnknown;
 
-    WinMarkFirstMissingRunningAggregateEvaluator(IScalarEvaluator[] argEvals) {
+    WinMarkFirstUnknownRunningAggregateEvaluator(ATypeTag unknownTypeTag, IScalarEvaluator[] argEvals) {
         this.argEvals = argEvals;
+        this.unknownTypeTag = unknownTypeTag.serialize();
         argValue = TaggedValuePointable.FACTORY.createPointable();
     }
 
@@ -76,19 +79,19 @@
 
     private boolean compute(IFrameTupleReference tuple) throws HyracksDataException {
         if (first) {
-            firstAllMissing = everyArgIsMissing(tuple);
+            firstAllUnknown = everyArgIsUnknown(tuple);
             first = false;
             return true;
         } else {
-            boolean thisAllMissing = firstAllMissing || everyArgIsMissing(tuple);
+            boolean thisAllMissing = firstAllUnknown || everyArgIsUnknown(tuple);
             return !thisAllMissing;
         }
     }
 
-    private boolean everyArgIsMissing(IFrameTupleReference tuple) throws HyracksDataException {
+    private boolean everyArgIsUnknown(IFrameTupleReference tuple) throws HyracksDataException {
         for (IScalarEvaluator argEval : argEvals) {
             argEval.evaluate(tuple, argValue);
-            if (argValue.getTag() != ATypeTag.SERIALIZED_MISSING_TYPE_TAG) {
+            if (argValue.getTag() != unknownTypeTag) {
                 return false;
             }
         }
diff --git a/asterixdb/asterix-runtime/src/test/java/org/apache/asterix/runtime/functions/ScalarAggregateTypeComputerTest.java b/asterixdb/asterix-runtime/src/test/java/org/apache/asterix/runtime/functions/ScalarAggregateTypeComputerTest.java
index cbde36c..a18b04f 100644
--- a/asterixdb/asterix-runtime/src/test/java/org/apache/asterix/runtime/functions/ScalarAggregateTypeComputerTest.java
+++ b/asterixdb/asterix-runtime/src/test/java/org/apache/asterix/runtime/functions/ScalarAggregateTypeComputerTest.java
@@ -221,7 +221,8 @@
         }
 
         @Override
-        public Object getVarType(LogicalVariable var, List<LogicalVariable> nonNullVariables,
+        public Object getVarType(LogicalVariable var, List<LogicalVariable> nonMissableVariables,
+                List<List<LogicalVariable>> correlatedMissableVariableLists, List<LogicalVariable> nonNullableVariables,
                 List<List<LogicalVariable>> correlatedNullableVariableLists) {
             throw new IllegalStateException();
         }
diff --git a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/AbstractCompilerFactoryBuilder.java b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/AbstractCompilerFactoryBuilder.java
index 3f03cc8..c22d54d 100644
--- a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/AbstractCompilerFactoryBuilder.java
+++ b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/AbstractCompilerFactoryBuilder.java
@@ -66,6 +66,7 @@
     protected IConflictingTypeResolver conflictingTypeResolver;
     protected IExpressionEvalSizeComputer expressionEvalSizeComputer;
     protected IMissingWriterFactory missingWriterFactory;
+    protected IMissingWriterFactory nullWriterFactory;
     protected IUnnestingPositionWriterFactory unnestingPositionWriterFactory;
     protected INormalizedKeyComputerFactoryProvider normalizedKeyComputerFactoryProvider;
     protected IPartialAggregationTypeComputer partialAggregationTypeComputer;
@@ -189,6 +190,14 @@
         return missingWriterFactory;
     }
 
+    public void setNullWriterFactory(IMissingWriterFactory nullWriterFactory) {
+        this.nullWriterFactory = nullWriterFactory;
+    }
+
+    public IMissingWriterFactory getNullWriterFactory() {
+        return nullWriterFactory;
+    }
+
     public void setUnnestingPositionWriterFactory(IUnnestingPositionWriterFactory unnestingPositionWriterFactory) {
         this.unnestingPositionWriterFactory = unnestingPositionWriterFactory;
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java
index c6a79a5..891980f 100644
--- a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java
+++ b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java
@@ -101,7 +101,7 @@
                         JobGenContext context = new JobGenContext(null, metadata, appContext,
                                 serializerDeserializerProvider, hashFunctionFactoryProvider, hashFunctionFamilyProvider,
                                 comparatorFactoryProvider, typeTraitProvider, binaryBooleanInspectorFactory,
-                                binaryIntegerInspectorFactory, printerProvider, missingWriterFactory,
+                                binaryIntegerInspectorFactory, printerProvider, missingWriterFactory, nullWriterFactory,
                                 unnestingPositionWriterFactory, normalizedKeyComputerFactoryProvider,
                                 expressionRuntimeProvider, expressionTypeComputer, oc, expressionEvalSizeComputer,
                                 partialAggregationTypeComputer, predEvaluatorFactoryProvider,
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ConstantExpression.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ConstantExpression.java
index c49f62f..e2a68da 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ConstantExpression.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ConstantExpression.java
@@ -57,6 +57,16 @@
         public String toString() {
             return "TRUE";
         }
+
+        @Override
+        public boolean equals(Object obj) {
+            return (obj instanceof IAlgebricksConstantValue) && ((IAlgebricksConstantValue) obj).isTrue();
+        }
+
+        @Override
+        public int hashCode() {
+            return Boolean.TRUE.hashCode();
+        }
     });
     public static final ConstantExpression FALSE = new ConstantExpression(new IAlgebricksConstantValue() {
 
@@ -84,6 +94,16 @@
         public String toString() {
             return "FALSE";
         }
+
+        @Override
+        public boolean equals(Object obj) {
+            return (obj instanceof IAlgebricksConstantValue) && ((IAlgebricksConstantValue) obj).isFalse();
+        }
+
+        @Override
+        public int hashCode() {
+            return Boolean.FALSE.hashCode();
+        }
     });
     public static final ConstantExpression NULL = new ConstantExpression(new IAlgebricksConstantValue() {
 
@@ -111,6 +131,16 @@
         public String toString() {
             return "NULL";
         }
+
+        @Override
+        public boolean equals(Object obj) {
+            return (obj instanceof IAlgebricksConstantValue) && ((IAlgebricksConstantValue) obj).isNull();
+        }
+
+        @Override
+        public int hashCode() {
+            return 0;
+        }
     });
     public static final ConstantExpression MISSING = new ConstantExpression(new IAlgebricksConstantValue() {
 
@@ -138,6 +168,16 @@
         public String toString() {
             return "MISSING";
         }
+
+        @Override
+        public boolean equals(Object obj) {
+            return (obj instanceof IAlgebricksConstantValue) && ((IAlgebricksConstantValue) obj).isMissing();
+        }
+
+        @Override
+        public int hashCode() {
+            return 0;
+        }
     });
 
     public ConstantExpression(IAlgebricksConstantValue value) {
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IMissableTypeComputer.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IMissableTypeComputer.java
index 029f41a..c18e072 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IMissableTypeComputer.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IMissableTypeComputer.java
@@ -18,12 +18,18 @@
  */
 package org.apache.hyracks.algebricks.core.algebra.expressions;
 
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-
 public interface IMissableTypeComputer {
-    public Object makeMissableType(Object type) throws AlgebricksException;
+    Object makeMissableType(Object type);
 
-    public boolean canBeMissing(Object type);
+    boolean canBeMissing(Object type);
 
-    public Object getNonOptionalType(Object type);
+    Object makeNullableType(Object type);
+
+    boolean canBeNull(Object type);
+
+    Object getNonOptionalType(Object type);
+
+    Object getNonMissableType(Object type);
+
+    Object getNonNullableType(Object type);
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IVariableTypeEnvironment.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IVariableTypeEnvironment.java
index e024147..d160e54 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IVariableTypeEnvironment.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IVariableTypeEnvironment.java
@@ -27,7 +27,8 @@
 public interface IVariableTypeEnvironment {
     public Object getVarType(LogicalVariable var) throws AlgebricksException;
 
-    public Object getVarType(LogicalVariable var, List<LogicalVariable> nonNullVariables,
+    public Object getVarType(LogicalVariable var, List<LogicalVariable> nonMissableVariables,
+            List<List<LogicalVariable>> correlatedMissableVariableLists, List<LogicalVariable> nonNullableVariables,
             List<List<LogicalVariable>> correlatedNullableVariableLists) throws AlgebricksException;
 
     public void setVarType(LogicalVariable var, Object type);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/EmptyTupleSourceOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/EmptyTupleSourceOperator.java
index 1af0ce4..1179dc4 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/EmptyTupleSourceOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/EmptyTupleSourceOperator.java
@@ -84,7 +84,9 @@
             }
 
             @Override
-            public Object getVarType(LogicalVariable var, List<LogicalVariable> nonNullVariables,
+            public Object getVarType(LogicalVariable var, List<LogicalVariable> nonMissableVariables,
+                    List<List<LogicalVariable>> correlatedMissableVariableLists,
+                    List<LogicalVariable> nonNullableVariables,
                     List<List<LogicalVariable>> correlatedNullableVariableLists) throws AlgebricksException {
                 return null;
             }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/LeftOuterJoinOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/LeftOuterJoinOperator.java
index 4e382d2..d709184 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/LeftOuterJoinOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/LeftOuterJoinOperator.java
@@ -27,24 +27,40 @@
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import org.apache.hyracks.algebricks.core.algebra.properties.LeftOuterTypePropagationPolicy;
 import org.apache.hyracks.algebricks.core.algebra.properties.TypePropagationPolicy;
 import org.apache.hyracks.algebricks.core.algebra.typing.ITypeEnvPointer;
 import org.apache.hyracks.algebricks.core.algebra.typing.ITypingContext;
 import org.apache.hyracks.algebricks.core.algebra.typing.OpRefTypeEnvPointer;
 import org.apache.hyracks.algebricks.core.algebra.typing.PropagatingTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import org.apache.hyracks.api.exceptions.ErrorCode;
 
 public class LeftOuterJoinOperator extends AbstractBinaryJoinOperator {
 
-    public LeftOuterJoinOperator(Mutable<ILogicalExpression> condition) {
+    private IAlgebricksConstantValue missingValue;
+
+    public LeftOuterJoinOperator(Mutable<ILogicalExpression> condition, IAlgebricksConstantValue missingValue) {
         super(JoinKind.LEFT_OUTER, condition);
+        setMissingValue(missingValue);
     }
 
     public LeftOuterJoinOperator(Mutable<ILogicalExpression> condition, Mutable<ILogicalOperator> input1,
-            Mutable<ILogicalOperator> input2) {
+            Mutable<ILogicalOperator> input2, IAlgebricksConstantValue missingValue) {
         super(JoinKind.LEFT_OUTER, condition, input1, input2);
+        setMissingValue(missingValue);
+    }
+
+    public IAlgebricksConstantValue getMissingValue() {
+        return missingValue;
+    }
+
+    public void setMissingValue(IAlgebricksConstantValue value) {
+        this.missingValue = validateMissingValue(value);
     }
 
     @Override
@@ -64,13 +80,40 @@
         for (int i = 0; i < n; i++) {
             envPointers[i] = new OpRefTypeEnvPointer(inputs.get(i), ctx);
         }
-        PropagatingTypeEnvironment env =
-                new PropagatingTypeEnvironment(ctx.getExpressionTypeComputer(), ctx.getMissableTypeComputer(),
-                        ctx.getMetadataProvider(), TypePropagationPolicy.LEFT_OUTER, envPointers);
+
+        TypePropagationPolicy typePropagationPolicy;
+        if (missingValue.isMissing()) {
+            typePropagationPolicy = LeftOuterTypePropagationPolicy.MISSABLE;
+        } else if (missingValue.isNull()) {
+            typePropagationPolicy = LeftOuterTypePropagationPolicy.NULLABLE;
+        } else {
+            throw new AlgebricksException(ErrorCode.ILLEGAL_STATE, String.valueOf(missingValue));
+        }
+        PropagatingTypeEnvironment env = new PropagatingTypeEnvironment(ctx.getExpressionTypeComputer(),
+                ctx.getMissableTypeComputer(), ctx.getMetadataProvider(), typePropagationPolicy, envPointers);
+
+        // live variables from right branch can be MISSING (or NULL) together
         List<LogicalVariable> liveVars = new ArrayList<>();
-        VariableUtilities.getLiveVariables(inputs.get(1).getValue(), liveVars); // live variables from right branch can be MISSING together
-        env.getCorrelatedMissableVariableLists().add(liveVars);
+        VariableUtilities.getLiveVariables(inputs.get(1).getValue(), liveVars);
+        if (missingValue.isMissing()) {
+            env.getCorrelatedMissableVariableLists().add(liveVars);
+        } else if (missingValue.isNull()) {
+            env.getCorrelatedNullableVariableLists().add(liveVars);
+        } else {
+            throw new AlgebricksException(ErrorCode.ILLEGAL_STATE, String.valueOf(missingValue));
+        }
         return env;
     }
 
+    private static IAlgebricksConstantValue validateMissingValue(IAlgebricksConstantValue value) {
+        if (value == null) {
+            throw new NullPointerException();
+        } else if (value.isMissing()) {
+            return ConstantExpression.MISSING.getValue();
+        } else if (value.isNull()) {
+            return ConstantExpression.NULL.getValue();
+        } else {
+            throw new IllegalArgumentException(String.valueOf(value));
+        }
+    }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/LeftOuterUnnestMapOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/LeftOuterUnnestMapOperator.java
index cd009c0..d91cf72 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/LeftOuterUnnestMapOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/LeftOuterUnnestMapOperator.java
@@ -25,10 +25,13 @@
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.typing.ITypingContext;
 import org.apache.hyracks.algebricks.core.algebra.typing.PropagatingTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import org.apache.hyracks.api.exceptions.ErrorCode;
 
 /**
  * Left-outer-unnest-map is similar to the unnest-map operator. The only
@@ -38,11 +41,21 @@
  */
 public class LeftOuterUnnestMapOperator extends AbstractUnnestMapOperator {
 
+    private IAlgebricksConstantValue missingValue;
+
     public LeftOuterUnnestMapOperator(List<LogicalVariable> variables, Mutable<ILogicalExpression> expression,
-            List<Object> variableTypes, boolean propagateInput) {
-        super(variables, expression, variableTypes, propagateInput);
+            List<Object> variableTypes, IAlgebricksConstantValue missingValue) {
         // propagateInput is always set to true for this operator.
-        this.propagateInput = true;
+        super(variables, expression, variableTypes, true);
+        setMissingValue(missingValue);
+    }
+
+    public IAlgebricksConstantValue getMissingValue() {
+        return missingValue;
+    }
+
+    public void setMissingValue(IAlgebricksConstantValue missingValue) {
+        this.missingValue = validateMissingValue(missingValue);
     }
 
     @Override
@@ -60,12 +73,32 @@
         // Propagates all input variables that come from the outer branch.
         PropagatingTypeEnvironment env = createPropagatingAllInputsTypeEnvironment(ctx);
 
-        // The produced variables of the this operator are missable because of the left outer semantics.
+        // The produced variables of the this operator are missable (or nullable) because of the left outer semantics.
         for (int i = 0; i < variables.size(); i++) {
-            env.setVarType(variables.get(i), ctx.getMissableTypeComputer().makeMissableType(variableTypes.get(i)));
+            Object varType = variableTypes.get(i);
+            Object outVarType;
+            if (missingValue.isMissing()) {
+                outVarType = ctx.getMissableTypeComputer().makeMissableType(varType);
+            } else if (missingValue.isNull()) {
+                outVarType = ctx.getMissableTypeComputer().makeNullableType(varType);
+            } else {
+                throw new AlgebricksException(ErrorCode.ILLEGAL_STATE, getSourceLocation(), "");
+            }
+            env.setVarType(variables.get(i), outVarType);
         }
 
         return env;
     }
 
+    private static IAlgebricksConstantValue validateMissingValue(IAlgebricksConstantValue value) {
+        if (value == null) {
+            throw new NullPointerException();
+        } else if (value.isMissing()) {
+            return ConstantExpression.MISSING.getValue();
+        } else if (value.isNull()) {
+            return ConstantExpression.NULL.getValue();
+        } else {
+            throw new IllegalArgumentException(String.valueOf(value));
+        }
+    }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/LeftOuterUnnestOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/LeftOuterUnnestOperator.java
index 14996dd..1e493ad 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/LeftOuterUnnestOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/LeftOuterUnnestOperator.java
@@ -19,20 +19,37 @@
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IMissableTypeComputer;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.typing.ITypingContext;
 import org.apache.hyracks.algebricks.core.algebra.typing.PropagatingTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import org.apache.hyracks.api.exceptions.ErrorCode;
 
 public class LeftOuterUnnestOperator extends AbstractUnnestNonMapOperator {
 
-    public LeftOuterUnnestOperator(LogicalVariable variable, Mutable<ILogicalExpression> expression) {
+    private IAlgebricksConstantValue missingValue;
+
+    public LeftOuterUnnestOperator(LogicalVariable variable, Mutable<ILogicalExpression> expression,
+            IAlgebricksConstantValue missingValue) {
         super(variable, expression);
+        setMissingValue(missingValue);
     }
 
     public LeftOuterUnnestOperator(LogicalVariable variable, Mutable<ILogicalExpression> expression,
-            LogicalVariable positionalVariable, Object positionalVariableType) {
+            LogicalVariable positionalVariable, Object positionalVariableType, IAlgebricksConstantValue missingValue) {
         super(variable, expression, positionalVariable, positionalVariableType);
+        setMissingValue(missingValue);
+    }
+
+    public IAlgebricksConstantValue getMissingValue() {
+        return missingValue;
+    }
+
+    public void setMissingValue(IAlgebricksConstantValue missingValue) {
+        this.missingValue = validateMissingValue(missingValue);
     }
 
     @Override
@@ -45,10 +62,26 @@
         PropagatingTypeEnvironment env = createPropagatingAllInputsTypeEnvironment(ctx);
 
         // The produced variables of the this operator are missable because of the left outer semantics.
+        Object outVarType, outPositionalVarType = null;
         Object t = env.getType(expression.getValue());
-        env.setVarType(variables.get(0), ctx.getMissableTypeComputer().makeMissableType(t));
+        IMissableTypeComputer tc = ctx.getMissableTypeComputer();
+        if (missingValue.isMissing()) {
+            outVarType = tc.makeMissableType(t);
+            if (positionalVariable != null) {
+                outPositionalVarType = tc.makeMissableType(positionalVariableType);
+            }
+        } else if (missingValue.isNull()) {
+            outVarType = tc.makeNullableType(t);
+            if (positionalVariable != null) {
+                outPositionalVarType = tc.makeNullableType(positionalVariableType);
+            }
+        } else {
+            throw new AlgebricksException(ErrorCode.ILLEGAL_STATE, getSourceLocation(), "");
+        }
+
+        env.setVarType(variables.get(0), outVarType);
         if (positionalVariable != null) {
-            env.setVarType(positionalVariable, ctx.getMissableTypeComputer().makeMissableType(positionalVariableType));
+            env.setVarType(positionalVariable, outPositionalVarType);
         }
 
         return env;
@@ -58,4 +91,16 @@
     public LogicalOperatorTag getOperatorTag() {
         return LogicalOperatorTag.LEFT_OUTER_UNNEST;
     }
+
+    private static IAlgebricksConstantValue validateMissingValue(IAlgebricksConstantValue value) {
+        if (value == null) {
+            throw new NullPointerException();
+        } else if (value.isMissing()) {
+            return ConstantExpression.MISSING.getValue();
+        } else if (value.isNull()) {
+            return ConstantExpression.NULL.getValue();
+        } else {
+            throw new IllegalArgumentException(String.valueOf(value));
+        }
+    }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/SelectOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/SelectOperator.java
index b2e2dfd..0b2c32d 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/SelectOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/SelectOperator.java
@@ -19,6 +19,7 @@
 package org.apache.hyracks.algebricks.core.algebra.operators.logical;
 
 import java.util.ArrayList;
+import java.util.List;
 
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -27,9 +28,12 @@
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
 import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.properties.TypePropagationPolicy;
 import org.apache.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
 import org.apache.hyracks.algebricks.core.algebra.typing.ITypeEnvPointer;
@@ -41,13 +45,28 @@
 
 public class SelectOperator extends AbstractLogicalOperator {
     private final Mutable<ILogicalExpression> condition;
-    private final boolean retainMissing;
+    private final IAlgebricksConstantValue retainMissingAsValue;
     private final LogicalVariable nullPlaceholderVar;
 
-    public SelectOperator(Mutable<ILogicalExpression> condition, boolean retainMissing,
+    public SelectOperator(Mutable<ILogicalExpression> condition) {
+        this(condition, null, null);
+    }
+
+    public SelectOperator(Mutable<ILogicalExpression> condition, IAlgebricksConstantValue retainMissingAsValue,
             LogicalVariable nullPlaceholderVar) {
         this.condition = condition;
-        this.retainMissing = retainMissing;
+        if (retainMissingAsValue == null) {
+            this.retainMissingAsValue = null;
+            if (nullPlaceholderVar != null) {
+                throw new IllegalArgumentException(nullPlaceholderVar.toString());
+            }
+        } else if (retainMissingAsValue.isMissing()) {
+            this.retainMissingAsValue = ConstantExpression.MISSING.getValue();
+        } else if (retainMissingAsValue.isNull()) {
+            this.retainMissingAsValue = ConstantExpression.NULL.getValue();
+        } else {
+            throw new IllegalArgumentException(retainMissingAsValue.toString());
+        }
         this.nullPlaceholderVar = nullPlaceholderVar;
     }
 
@@ -60,11 +79,11 @@
         return condition;
     }
 
-    public boolean getRetainMissing() {
-        return retainMissing;
+    public IAlgebricksConstantValue getRetainMissingAsValue() {
+        return retainMissingAsValue;
     }
 
-    public LogicalVariable getMissingPlaceholderVariable() throws AlgebricksException {
+    public LogicalVariable getMissingPlaceholderVariable() {
         return nullPlaceholderVar;
     }
 
@@ -109,12 +128,11 @@
         ILogicalExpression a1 = f1.getArguments().get(0).getValue();
         if (a1.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
             AbstractFunctionCallExpression f2 = (AbstractFunctionCallExpression) a1;
-            if (f2.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.IS_MISSING)) {
-                ILogicalExpression a2 = f2.getArguments().get(0).getValue();
-                if (a2.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
-                    LogicalVariable var = ((VariableReferenceExpression) a2).getVariableReference();
-                    env.getNonMissableVariables().add(var);
-                }
+            FunctionIdentifier f2id = f2.getFunctionIdentifier();
+            if (f2id.equals(AlgebricksBuiltinFunctions.IS_MISSING)) {
+                extractFunctionArgVarInto(f2, env.getNonMissableVariables());
+            } else if (f2id.equals(AlgebricksBuiltinFunctions.IS_NULL)) {
+                extractFunctionArgVarInto(f2, env.getNonNullableVariables());
             }
         }
         return env;
@@ -124,4 +142,13 @@
     public boolean requiresVariableReferenceExpressions() {
         return false;
     }
+
+    private static void extractFunctionArgVarInto(AbstractFunctionCallExpression callExpr,
+            List<? super LogicalVariable> outList) {
+        ILogicalExpression arg = callExpr.getArguments().get(0).getValue();
+        if (arg.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+            LogicalVariable var = ((VariableReferenceExpression) arg).getVariableReference();
+            outList.add(var);
+        }
+    }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
index aa5b5b4..e5d0241 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
@@ -225,6 +225,9 @@
             return Boolean.FALSE;
         }
         LeftOuterJoinOperator joinOpArg = (LeftOuterJoinOperator) copyAndSubstituteVar(op, arg);
+        if (!op.getMissingValue().equals(joinOpArg.getMissingValue())) {
+            return Boolean.FALSE;
+        }
         boolean isomorphic = op.getCondition().getValue().equals(joinOpArg.getCondition().getValue());
         return isomorphic;
     }
@@ -451,6 +454,10 @@
         if (!isomorphic) {
             return Boolean.FALSE;
         }
+        isomorphic = op.getMissingValue().equals(loUnnestOpArg.getMissingValue());
+        if (!isomorphic) {
+            return Boolean.FALSE;
+        }
         isomorphic = op.getExpressionRef().getValue().equals(loUnnestOpArg.getExpressionRef().getValue());
         return isomorphic;
     }
@@ -818,6 +825,10 @@
         if (!isomorphic) {
             return Boolean.FALSE;
         }
+        isomorphic = op.getMissingValue().equals(unnestOpArg.getMissingValue());
+        if (!isomorphic) {
+            return Boolean.FALSE;
+        }
         isomorphic = op.getExpressionRef().getValue().equals(unnestOpArg.getExpressionRef().getValue());
         return isomorphic;
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java
index e05d12a..199c2e1 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java
@@ -389,7 +389,7 @@
         LeftOuterJoinOperator opCopy =
                 new LeftOuterJoinOperator(exprDeepCopyVisitor.deepCopyExpressionReference(op.getCondition()),
                         deepCopyOperatorReference(op.getInputs().get(0), arg),
-                        deepCopyOperatorReference(op.getInputs().get(1), arg));
+                        deepCopyOperatorReference(op.getInputs().get(1), arg), op.getMissingValue());
         copyAnnotations(op, opCopy);
         copySourceLocation(op, opCopy);
         opCopy.setExecutionMode(op.getExecutionMode());
@@ -474,7 +474,7 @@
     @Override
     public ILogicalOperator visitSelectOperator(SelectOperator op, ILogicalOperator arg) throws AlgebricksException {
         SelectOperator opCopy = new SelectOperator(exprDeepCopyVisitor.deepCopyExpressionReference(op.getCondition()),
-                op.getRetainMissing(), deepCopyVariable(op.getMissingPlaceholderVariable()));
+                op.getRetainMissingAsValue(), deepCopyVariable(op.getMissingPlaceholderVariable()));
         deepCopyInputsAnnotationsAndExecutionMode(op, arg, opCopy);
         return opCopy;
     }
@@ -544,7 +544,7 @@
             throws AlgebricksException {
         LeftOuterUnnestMapOperator opCopy = new LeftOuterUnnestMapOperator(deepCopyVariableList(op.getVariables()),
                 exprDeepCopyVisitor.deepCopyExpressionReference(op.getExpressionRef()), op.getVariableTypes(),
-                op.propagatesInput());
+                op.getMissingValue());
         deepCopyInputsAnnotationsAndExecutionMode(op, arg, opCopy);
         return opCopy;
     }
@@ -590,7 +590,7 @@
             throws AlgebricksException {
         LeftOuterUnnestOperator opCopy = new LeftOuterUnnestOperator(deepCopyVariable(op.getVariable()),
                 exprDeepCopyVisitor.deepCopyExpressionReference(op.getExpressionRef()),
-                deepCopyVariable(op.getPositionalVariable()), op.getPositionalVariableType());
+                deepCopyVariable(op.getPositionalVariable()), op.getPositionalVariableType(), op.getMissingValue());
         deepCopyInputsAnnotationsAndExecutionMode(op, arg, opCopy);
         return opCopy;
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
index 1c91d7b..7b67af1 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
@@ -133,7 +133,7 @@
     @Override
     public ILogicalOperator visitLeftOuterJoinOperator(LeftOuterJoinOperator op, Void arg) throws AlgebricksException {
         return new LeftOuterJoinOperator(deepCopyExpressionRef(op.getCondition()), op.getInputs().get(0),
-                op.getInputs().get(1));
+                op.getInputs().get(1), op.getMissingValue());
     }
 
     @Override
@@ -158,7 +158,7 @@
 
     @Override
     public ILogicalOperator visitSelectOperator(SelectOperator op, Void arg) throws AlgebricksException {
-        return new SelectOperator(deepCopyExpressionRef(op.getCondition()), op.getRetainMissing(),
+        return new SelectOperator(deepCopyExpressionRef(op.getCondition()), op.getRetainMissingAsValue(),
                 op.getMissingPlaceholderVariable());
     }
 
@@ -247,7 +247,7 @@
         ArrayList<LogicalVariable> newInputList = new ArrayList<>();
         newInputList.addAll(op.getVariables());
         return new LeftOuterUnnestMapOperator(newInputList, deepCopyExpressionRef(op.getExpressionRef()),
-                new ArrayList<>(op.getVariableTypes()), op.propagatesInput());
+                new ArrayList<>(op.getVariableTypes()), op.getMissingValue());
     }
 
     @Override
@@ -420,7 +420,7 @@
     public ILogicalOperator visitLeftOuterUnnestOperator(LeftOuterUnnestOperator op, Void arg)
             throws AlgebricksException {
         return new LeftOuterUnnestOperator(op.getVariable(), deepCopyExpressionRef(op.getExpressionRef()),
-                op.getPositionalVariable(), op.getPositionalVariableType());
+                op.getPositionalVariable(), op.getPositionalVariableType(), op.getMissingValue());
     }
 
     @Override
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractUnnestPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractUnnestPOperator.java
index 90e88dc..45360d4 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractUnnestPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractUnnestPOperator.java
@@ -31,11 +31,13 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractUnnestNonMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterUnnestOperator;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
 import org.apache.hyracks.algebricks.data.IUnnestingPositionWriterFactory;
 import org.apache.hyracks.algebricks.runtime.base.IUnnestingEvaluatorFactory;
 import org.apache.hyracks.algebricks.runtime.operators.std.UnnestRuntimeFactory;
+import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.util.LogRedactionUtil;
 
@@ -84,8 +86,11 @@
         int[] projectionList = JobGenHelper.projectAllVariables(opSchema);
         IUnnestingPositionWriterFactory positionWriterFactory =
                 unnest.hasPositionalVariable() ? context.getUnnestingPositionWriterFactory() : null;
+        IMissingWriterFactory missingWriterFactory = leftOuter
+                ? JobGenHelper.getMissingWriterFactory(context, ((LeftOuterUnnestOperator) op).getMissingValue())
+                : null;
         UnnestRuntimeFactory unnestRuntime = new UnnestRuntimeFactory(outCol, unnestingFactory, projectionList,
-                positionWriterFactory, leftOuter, context.getMissingWriterFactory());
+                positionWriterFactory, leftOuter, missingWriterFactory);
         unnestRuntime.setSourceLocation(unnest.getSourceLocation());
         RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
         builder.contributeMicroOperator(unnest, unnestRuntime, recDesc);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/HybridHashJoinPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/HybridHashJoinPOperator.java
index c7c7588..7e7d012 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/HybridHashJoinPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/HybridHashJoinPOperator.java
@@ -36,6 +36,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator.JoinKind;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
 import org.apache.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
 import org.apache.hyracks.algebricks.core.algebra.properties.LocalGroupingProperty;
@@ -128,9 +129,9 @@
         IOperatorDescriptorRegistry spec = builder.getJobSpec();
         IOperatorDescriptor opDesc;
 
-        opDesc = generateOptimizedHashJoinRuntime(context, inputSchemas, keysLeft, keysRight, leftHashFunFamilies,
-                rightHashFunFamilies, comparatorFactory, reverseComparatorFactory, predEvaluatorFactory, recDescriptor,
-                spec);
+        opDesc = generateOptimizedHashJoinRuntime(context, joinOp, inputSchemas, keysLeft, keysRight,
+                leftHashFunFamilies, rightHashFunFamilies, comparatorFactory, reverseComparatorFactory,
+                predEvaluatorFactory, recDescriptor, spec);
         opDesc.setSourceLocation(op.getSourceLocation());
         contributeOpDesc(builder, (AbstractLogicalOperator) op, opDesc);
 
@@ -140,11 +141,12 @@
         builder.contributeGraphEdge(src2, 0, op, 1);
     }
 
-    private IOperatorDescriptor generateOptimizedHashJoinRuntime(JobGenContext context, IOperatorSchema[] inputSchemas,
-            int[] keysLeft, int[] keysRight, IBinaryHashFunctionFamily[] leftHashFunFamilies,
-            IBinaryHashFunctionFamily[] rightHashFunFamilies, ITuplePairComparatorFactory comparatorFactory,
-            ITuplePairComparatorFactory reverseComparatorFactory, IPredicateEvaluatorFactory predEvaluatorFactory,
-            RecordDescriptor recDescriptor, IOperatorDescriptorRegistry spec) {
+    private IOperatorDescriptor generateOptimizedHashJoinRuntime(JobGenContext context,
+            AbstractBinaryJoinOperator joinOp, IOperatorSchema[] inputSchemas, int[] keysLeft, int[] keysRight,
+            IBinaryHashFunctionFamily[] leftHashFunFamilies, IBinaryHashFunctionFamily[] rightHashFunFamilies,
+            ITuplePairComparatorFactory comparatorFactory, ITuplePairComparatorFactory reverseComparatorFactory,
+            IPredicateEvaluatorFactory predEvaluatorFactory, RecordDescriptor recDescriptor,
+            IOperatorDescriptorRegistry spec) throws AlgebricksException {
         int memSizeInFrames = localMemoryRequirements.getMemoryBudgetInFrames();
         switch (kind) {
             case INNER:
@@ -152,10 +154,8 @@
                         getFudgeFactor(), keysLeft, keysRight, leftHashFunFamilies, rightHashFunFamilies, recDescriptor,
                         comparatorFactory, reverseComparatorFactory, predEvaluatorFactory);
             case LEFT_OUTER:
-                IMissingWriterFactory[] nonMatchWriterFactories = new IMissingWriterFactory[inputSchemas[1].getSize()];
-                for (int j = 0; j < nonMatchWriterFactories.length; j++) {
-                    nonMatchWriterFactories[j] = context.getMissingWriterFactory();
-                }
+                IMissingWriterFactory[] nonMatchWriterFactories = JobGenHelper.createMissingWriterFactories(context,
+                        ((LeftOuterJoinOperator) joinOp).getMissingValue(), inputSchemas[1].getSize());
                 return new OptimizedHybridHashJoinOperatorDescriptor(spec, memSizeInFrames, maxInputBuildSizeInFrames,
                         getFudgeFactor(), keysLeft, keysRight, leftHashFunFamilies, rightHashFunFamilies, recDescriptor,
                         comparatorFactory, reverseComparatorFactory, predEvaluatorFactory, true,
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InMemoryHashJoinPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InMemoryHashJoinPOperator.java
index 8f911b4..84dda11 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InMemoryHashJoinPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InMemoryHashJoinPOperator.java
@@ -34,6 +34,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator.JoinKind;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
 import org.apache.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
@@ -119,10 +120,8 @@
                         memSizeInFrames);
                 break;
             case LEFT_OUTER:
-                IMissingWriterFactory[] nonMatchWriterFactories = new IMissingWriterFactory[inputSchemas[1].getSize()];
-                for (int j = 0; j < nonMatchWriterFactories.length; j++) {
-                    nonMatchWriterFactories[j] = context.getMissingWriterFactory();
-                }
+                IMissingWriterFactory[] nonMatchWriterFactories = JobGenHelper.createMissingWriterFactories(context,
+                        ((LeftOuterJoinOperator) joinOp).getMissingValue(), inputSchemas[1].getSize());
                 opDesc = new InMemoryHashJoinOperatorDescriptor(spec, keysLeft, keysRight, leftHashFunFactories,
                         rightHashFunFactories, comparatorFactory, predEvaluatorFactory, recDescriptor, true,
                         nonMatchWriterFactories, tableSize, memSizeInFrames);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/NestedLoopJoinPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/NestedLoopJoinPOperator.java
index 0991db8..6c5c8d0 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/NestedLoopJoinPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/NestedLoopJoinPOperator.java
@@ -29,6 +29,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator.JoinKind;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.properties.BroadcastPartitioningProperty;
 import org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
 import org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
@@ -135,10 +136,8 @@
                         null);
                 break;
             case LEFT_OUTER:
-                IMissingWriterFactory[] nonMatchWriterFactories = new IMissingWriterFactory[inputSchemas[1].getSize()];
-                for (int j = 0; j < nonMatchWriterFactories.length; j++) {
-                    nonMatchWriterFactories[j] = context.getMissingWriterFactory();
-                }
+                IMissingWriterFactory[] nonMatchWriterFactories = JobGenHelper.createMissingWriterFactories(context,
+                        ((LeftOuterJoinOperator) join).getMissingValue(), inputSchemas[1].getSize());
                 opDesc = new NestedLoopJoinOperatorDescriptor(spec, comparatorFactory, recDescriptor, memSize, true,
                         nonMatchWriterFactories);
                 break;
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StreamSelectPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StreamSelectPOperator.java
index a519275..5af7f98 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StreamSelectPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/StreamSelectPOperator.java
@@ -23,6 +23,7 @@
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
 import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionRuntimeProvider;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
@@ -32,6 +33,7 @@
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.algebricks.runtime.operators.std.StreamSelectRuntimeFactory;
+import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 
 public class StreamSelectPOperator extends AbstractPhysicalOperator {
@@ -66,10 +68,13 @@
         IExpressionRuntimeProvider expressionRuntimeProvider = context.getExpressionRuntimeProvider();
         IScalarEvaluatorFactory cond = expressionRuntimeProvider.createEvaluatorFactory(
                 select.getCondition().getValue(), context.getTypeEnvironment(op), inputSchemas, context);
+        IAlgebricksConstantValue retainMissingAsValue = select.getRetainMissingAsValue();
+        boolean retainMissing = retainMissingAsValue != null;
+        IMissingWriterFactory missingWriterFactory =
+                retainMissing ? JobGenHelper.getMissingWriterFactory(context, retainMissingAsValue) : null;
         StreamSelectRuntimeFactory runtime =
-                new StreamSelectRuntimeFactory(cond, null, context.getBinaryBooleanInspectorFactory(),
-                        select.getRetainMissing(), inputSchemas[0].findVariable(select.getMissingPlaceholderVariable()),
-                        context.getMissingWriterFactory());
+                new StreamSelectRuntimeFactory(cond, null, context.getBinaryBooleanInspectorFactory(), retainMissing,
+                        inputSchemas[0].findVariable(select.getMissingPlaceholderVariable()), missingWriterFactory);
         runtime.setSourceLocation(select.getSourceLocation());
         // contribute one Asterix framewriter
         RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
index 625418a..64abaee 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
@@ -30,6 +30,7 @@
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
 import org.apache.hyracks.algebricks.core.algebra.base.IPhysicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionInfo;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
@@ -180,8 +181,9 @@
 
     @Override
     public Void visitLeftOuterJoinOperator(LeftOuterJoinOperator op, Integer indent) throws AlgebricksException {
-        addIndent(indent).append("left outer join (").append(op.getCondition().getValue().accept(exprVisitor, indent))
-                .append(")");
+        addIndent(indent).append("left outer join ").append("(")
+                .append(op.getCondition().getValue().accept(exprVisitor, indent)).append(")")
+                .append(printLeftOuterMissingValue(op.getMissingValue()));
         return null;
     }
 
@@ -244,8 +246,11 @@
 
     @Override
     public Void visitSelectOperator(SelectOperator op, Integer indent) throws AlgebricksException {
+        String retainMissing = op.getMissingPlaceholderVariable() != null
+                ? " retain-untrue (" + op.getMissingPlaceholderVariable() + " <- " + op.getRetainMissingAsValue() + ")"
+                : "";
         addIndent(indent).append("select (").append(op.getCondition().getValue().accept(exprVisitor, indent))
-                .append(")");
+                .append(")").append(retainMissing);
         return null;
     }
 
@@ -306,17 +311,18 @@
 
     @Override
     public Void visitLeftOuterUnnestOperator(LeftOuterUnnestOperator op, Integer indent) throws AlgebricksException {
-        addIndent(indent).append("outer-unnest " + op.getVariable());
+        addIndent(indent).append("outer-unnest ").append(String.valueOf(op.getVariable()));
         if (op.getPositionalVariable() != null) {
             buffer.append(" at " + op.getPositionalVariable());
         }
         buffer.append(" <- " + op.getExpressionRef().getValue().accept(exprVisitor, indent));
+        buffer.append(printLeftOuterMissingValue(op.getMissingValue()));
         return null;
     }
 
     @Override
     public Void visitUnnestMapOperator(UnnestMapOperator op, Integer indent) throws AlgebricksException {
-        AlgebricksStringBuilderWriter plan = printAbstractUnnestMapOperator(op, indent, "unnest-map");
+        AlgebricksStringBuilderWriter plan = printAbstractUnnestMapOperator(op, indent, "unnest-map", null);
         appendSelectConditionInformation(plan, op.getSelectCondition(), indent);
         appendLimitInformation(plan, op.getOutputLimit());
         return null;
@@ -325,14 +331,16 @@
     @Override
     public Void visitLeftOuterUnnestMapOperator(LeftOuterUnnestMapOperator op, Integer indent)
             throws AlgebricksException {
-        printAbstractUnnestMapOperator(op, indent, "left-outer-unnest-map");
+        printAbstractUnnestMapOperator(op, indent, "left-outer-unnest-map", op.getMissingValue());
         return null;
     }
 
     private AlgebricksStringBuilderWriter printAbstractUnnestMapOperator(AbstractUnnestMapOperator op, Integer indent,
-            String opSignature) throws AlgebricksException {
-        AlgebricksStringBuilderWriter plan = addIndent(indent).append(opSignature + " " + op.getVariables() + " <- "
-                + op.getExpressionRef().getValue().accept(exprVisitor, indent));
+            String opSignature, IAlgebricksConstantValue leftOuterMissingValue) throws AlgebricksException {
+        AlgebricksStringBuilderWriter plan =
+                addIndent(indent).append(opSignature).append(' ').append(String.valueOf(op.getVariables()))
+                        .append(" <- ").append(op.getExpressionRef().getValue().accept(exprVisitor, indent)).append(
+                                leftOuterMissingValue != null ? printLeftOuterMissingValue(leftOuterMissingValue) : "");
         appendFilterInformation(plan, op.getMinFilterVars(), op.getMaxFilterVars());
         return plan;
     }
@@ -665,4 +673,8 @@
             }
         }
     }
+
+    private static String printLeftOuterMissingValue(IAlgebricksConstantValue leftOuterMissingValue) {
+        return leftOuterMissingValue.isNull() ? " (or <- " + leftOuterMissingValue + ")" : "";
+    }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
index ed5299c..2728e46 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
@@ -34,6 +34,7 @@
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
 import org.apache.hyracks.algebricks.core.algebra.base.IPhysicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IProjectionInfo;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
@@ -94,6 +95,7 @@
     private static final String EXPRESSIONS_FIELD = "expressions";
     private static final String EXPRESSION_FIELD = "expression";
     private static final String CONDITION_FIELD = "condition";
+    private static final String MISSING_VALUE_FIELD = "missing-value";
 
     private final Map<AbstractLogicalOperator, String> operatorIdentity = new HashMap<>();
     private final IdCounter idCounter = new IdCounter();
@@ -301,6 +303,9 @@
         try {
             jsonGenerator.writeStringField(OPERATOR_FIELD, "left-outer-join");
             writeStringFieldExpression(CONDITION_FIELD, op.getCondition(), indent);
+            if (op.getMissingValue().isNull()) {
+                writeNullField(MISSING_VALUE_FIELD);
+            }
             return null;
         } catch (IOException e) {
             throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
@@ -467,14 +472,21 @@
 
     @Override
     public Void visitLeftOuterUnnestOperator(LeftOuterUnnestOperator op, Void indent) throws AlgebricksException {
-        writeUnnestNonMapOperator(op, "outer-unnest", indent);
-        return null;
+        try {
+            writeUnnestNonMapOperator(op, "outer-unnest", indent);
+            if (op.getMissingValue().isNull()) {
+                writeNullField(MISSING_VALUE_FIELD);
+            }
+            return null;
+        } catch (IOException e) {
+            throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
+        }
     }
 
     @Override
     public Void visitUnnestMapOperator(UnnestMapOperator op, Void indent) throws AlgebricksException {
         try {
-            writeUnnestMapOperator(op, indent, "unnest-map");
+            writeUnnestMapOperator(op, indent, "unnest-map", null);
             writeSelectLimitInformation(op.getSelectCondition(), op.getOutputLimit(), indent);
             return null;
         } catch (IOException e) {
@@ -484,7 +496,7 @@
 
     @Override
     public Void visitLeftOuterUnnestMapOperator(LeftOuterUnnestMapOperator op, Void indent) throws AlgebricksException {
-        writeUnnestMapOperator(op, indent, "left-outer-unnest-map");
+        writeUnnestMapOperator(op, indent, "left-outer-unnest-map", op.getMissingValue());
         return null;
     }
 
@@ -782,8 +794,8 @@
         }
     }
 
-    private void writeUnnestMapOperator(AbstractUnnestMapOperator op, Void indent, String opName)
-            throws AlgebricksException {
+    private void writeUnnestMapOperator(AbstractUnnestMapOperator op, Void indent, String opName,
+            IAlgebricksConstantValue leftOuterMissingValue) throws AlgebricksException {
         try {
             jsonGenerator.writeStringField(OPERATOR_FIELD, opName);
             List<LogicalVariable> variables = op.getVariables();
@@ -792,6 +804,9 @@
             }
             writeArrayFieldOfExpression(EXPRESSIONS_FIELD, op.getExpressionRef(), indent);
             writeFilterInformation(op.getMinFilterVars(), op.getMaxFilterVars());
+            if (leftOuterMissingValue != null && leftOuterMissingValue.isNull()) {
+                writeNullField(MISSING_VALUE_FIELD);
+            }
         } catch (IOException e) {
             throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
         }
@@ -963,6 +978,13 @@
         jsonGenerator.writeEndObject();
     }
 
+    /////////////// other fields ///////////////
+
+    /** Writes "fieldName": null */
+    private void writeNullField(String fieldName) throws IOException {
+        jsonGenerator.writeNullField(fieldName);
+    }
+
     private void flushContentToWriter() throws AlgebricksException {
         try {
             jsonGenerator.flush();
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/LeftOuterTypePropagationPolicy.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/LeftOuterTypePropagationPolicy.java
new file mode 100644
index 0000000..07f1f7f
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/LeftOuterTypePropagationPolicy.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.algebricks.core.algebra.properties;
+
+import java.util.List;
+
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IMissableTypeComputer;
+import org.apache.hyracks.algebricks.core.algebra.typing.ITypeEnvPointer;
+import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+
+public abstract class LeftOuterTypePropagationPolicy extends TypePropagationPolicy {
+
+    public static final TypePropagationPolicy MISSABLE = new LeftOuterTypePropagationPolicy() {
+
+        protected Object computeInnerBranchVarType(LogicalVariable var, Object varType, IMissableTypeComputer ntc,
+                List<LogicalVariable> nonMissableVariableList,
+                List<List<LogicalVariable>> correlatedMissableVariableLists,
+                List<LogicalVariable> nonNullableVariableList,
+                List<List<LogicalVariable>> correlatedNullableVariableLists) {
+            boolean makeMissable =
+                    !inCorrelatedVariableList(var, correlatedMissableVariableLists, nonMissableVariableList);
+            return makeMissable ? ntc.makeMissableType(varType) : varType;
+        }
+    };
+
+    public static final TypePropagationPolicy NULLABLE = new LeftOuterTypePropagationPolicy() {
+
+        protected Object computeInnerBranchVarType(LogicalVariable var, Object varType, IMissableTypeComputer ntc,
+                List<LogicalVariable> nonMissableVariableList,
+                List<List<LogicalVariable>> correlatedMissableVariableLists,
+                List<LogicalVariable> nonNullableVariableList,
+                List<List<LogicalVariable>> correlatedNullableVariableLists) {
+            boolean makeNullable =
+                    !inCorrelatedVariableList(var, correlatedNullableVariableLists, nonNullableVariableList);
+            return makeNullable ? ntc.makeNullableType(varType) : varType;
+        }
+    };
+
+    @Override
+    public Object getVarType(LogicalVariable var, IMissableTypeComputer ntc,
+            List<LogicalVariable> nonMissableVariableList, List<List<LogicalVariable>> correlatedMissableVariableLists,
+            List<LogicalVariable> nonNullableVariableList, List<List<LogicalVariable>> correlatedNullableVariableLists,
+            ITypeEnvPointer... typeEnvs) throws AlgebricksException {
+        int n = typeEnvs.length;
+        // Searches from the inner branch to the outer branch.
+        // TODO(buyingyi): A split operator could lead to the case that the type for a variable could be
+        // found in both inner and outer branches. Fix computeOutputTypeEnvironment() in ProjectOperator
+        // and investigate why many test queries fail if only live variables' types are propagated.
+        for (int i = n - 1; i >= 0; i--) {
+            Object varType = typeEnvs[i].getTypeEnv().getVarType(var, nonMissableVariableList,
+                    correlatedMissableVariableLists, nonNullableVariableList, correlatedNullableVariableLists);
+            if (varType == null) {
+                continue;
+            }
+            if (i == 0) { // outer branch
+                return varType;
+            }
+            // inner branch
+            return computeInnerBranchVarType(var, varType, ntc, nonMissableVariableList,
+                    correlatedMissableVariableLists, nonNullableVariableList, correlatedNullableVariableLists);
+        }
+        return null;
+    }
+
+    protected abstract Object computeInnerBranchVarType(LogicalVariable var, Object varType, IMissableTypeComputer ntc,
+            List<LogicalVariable> nonMissableVariableList, List<List<LogicalVariable>> correlatedMissableVariableLists,
+            List<LogicalVariable> nonNullableVariableList, List<List<LogicalVariable>> correlatedNullableVariableLists);
+
+    protected static boolean inCorrelatedVariableList(LogicalVariable var,
+            List<List<LogicalVariable>> correlatedOptionalVariableLists,
+            List<LogicalVariable> nonOptionalVariableList) {
+        if (!nonOptionalVariableList.isEmpty()) {
+            for (List<LogicalVariable> correlatedVariables : correlatedOptionalVariableLists) {
+                if (correlatedVariables.contains(var)
+                        && !OperatorPropertiesUtil.disjoint(correlatedVariables, nonOptionalVariableList)) {
+                    return true;
+                }
+            }
+        }
+        return false;
+    }
+};
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/TypePropagationPolicy.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/TypePropagationPolicy.java
index c37c674..9cd5fe1 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/TypePropagationPolicy.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/TypePropagationPolicy.java
@@ -32,7 +32,9 @@
         @Override
         public Object getVarType(LogicalVariable var, IMissableTypeComputer ntc,
                 List<LogicalVariable> nonMissableVariableList,
-                List<List<LogicalVariable>> correlatedMissableVariableLists, ITypeEnvPointer... typeEnvs)
+                List<List<LogicalVariable>> correlatedMissableVariableLists,
+                List<LogicalVariable> nonNullableVariableList,
+                List<List<LogicalVariable>> correlatedNullableVariableLists, ITypeEnvPointer... typeEnvs)
                 throws AlgebricksException {
             for (ITypeEnvPointer p : typeEnvs) {
                 IVariableTypeEnvironment env = p.getTypeEnv();
@@ -40,67 +42,22 @@
                     throw new AlgebricksException(
                             "Null environment for pointer " + p + " in getVarType for var=" + var);
                 }
-                Object t = env.getVarType(var, nonMissableVariableList, correlatedMissableVariableLists);
+                Object t = env.getVarType(var, nonMissableVariableList, correlatedMissableVariableLists,
+                        nonNullableVariableList, correlatedNullableVariableLists);
                 if (t != null) {
-                    if (ntc != null && ntc.canBeMissing(t)) {
-                        for (List<LogicalVariable> list : correlatedMissableVariableLists) {
-                            if (list.contains(var)) {
-                                for (LogicalVariable v : list) {
-                                    if (nonMissableVariableList.contains(v)) {
-                                        return ntc.getNonOptionalType(t);
-                                    }
-                                }
-                            }
-                        }
+                    boolean makeNonMissable = !nonMissableVariableList.isEmpty() && ntc.canBeMissing(t)
+                            && nonMissableVariableList.contains(var);
+                    boolean makeNonNullable = !nonNullableVariableList.isEmpty() && ntc.canBeNull(t)
+                            && nonNullableVariableList.contains(var);
+                    if (makeNonMissable && makeNonNullable) {
+                        return ntc.getNonOptionalType(t);
+                    } else if (makeNonMissable) {
+                        return ntc.getNonMissableType(t);
+                    } else if (makeNonNullable) {
+                        return ntc.getNonNullableType(t);
+                    } else {
+                        return t;
                     }
-                    return t;
-                }
-            }
-            return null;
-        }
-    };
-
-    public static final TypePropagationPolicy LEFT_OUTER = new TypePropagationPolicy() {
-
-        @Override
-        public Object getVarType(LogicalVariable var, IMissableTypeComputer ntc,
-                List<LogicalVariable> nonMissableVariableList,
-                List<List<LogicalVariable>> correlatedMissableVariableLists, ITypeEnvPointer... typeEnvs)
-                throws AlgebricksException {
-            int n = typeEnvs.length;
-            // Searches from the inner branch to the outer branch.
-            // TODO(buyingyi): A split operator could lead to the case that the type for a variable could be
-            // found in both inner and outer branches. Fix computeOutputTypeEnvironment() in ProjectOperator
-            // and investigate why many test queries fail if only live variables' types are propagated.
-            for (int i = n - 1; i >= 0; i--) {
-                Object t = typeEnvs[i].getTypeEnv().getVarType(var, nonMissableVariableList,
-                        correlatedMissableVariableLists);
-                if (t == null) {
-                    continue;
-                }
-                if (i == 0) { // outer branch
-                    return t;
-                }
-
-                // inner branch
-                boolean nonMissingVarIsProduced = false;
-                for (LogicalVariable v : nonMissableVariableList) {
-                    boolean toBreak = false;
-                    if (v == var) {
-                        nonMissingVarIsProduced = true;
-                        toBreak = true;
-                    } else if (typeEnvs[i].getTypeEnv().getVarType(v) != null) {
-                        nonMissingVarIsProduced = true;
-                        toBreak = true;
-                    }
-                    if (toBreak) {
-                        break;
-                    }
-                }
-                if (nonMissingVarIsProduced) {
-                    return t;
-                } else {
-                    return ntc.makeMissableType(t);
                 }
             }
             return null;
@@ -109,5 +66,6 @@
 
     public abstract Object getVarType(LogicalVariable var, IMissableTypeComputer ntc,
             List<LogicalVariable> nonMissableVariableList, List<List<LogicalVariable>> correlatedMissableVariableLists,
+            List<LogicalVariable> nonNullableVariableList, List<List<LogicalVariable>> correlatedNullableVariableLists,
             ITypeEnvPointer... typeEnvs) throws AlgebricksException;
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/typing/NonPropagatingTypeEnvironment.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/typing/NonPropagatingTypeEnvironment.java
index 547737c..3bd4bc3 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/typing/NonPropagatingTypeEnvironment.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/typing/NonPropagatingTypeEnvironment.java
@@ -38,7 +38,8 @@
     }
 
     @Override
-    public Object getVarType(LogicalVariable var, List<LogicalVariable> nonNullVariables,
+    public Object getVarType(LogicalVariable var, List<LogicalVariable> nonMissableVariables,
+            List<List<LogicalVariable>> correlatedMissableVariableLists, List<LogicalVariable> nonNullableVariables,
             List<List<LogicalVariable>> correlatedNullableVariableLists) throws AlgebricksException {
         return getVarType(var);
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/typing/PropagateOperatorInputsTypeEnvironment.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/typing/PropagateOperatorInputsTypeEnvironment.java
deleted file mode 100644
index 1a85768..0000000
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/typing/PropagateOperatorInputsTypeEnvironment.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.hyracks.algebricks.core.algebra.typing;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.commons.lang3.mutable.Mutable;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
-import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
-import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionTypeComputer;
-import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
-import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
-
-public class PropagateOperatorInputsTypeEnvironment extends AbstractTypeEnvironment {
-
-    private final List<LogicalVariable> nonNullVariables = new ArrayList<LogicalVariable>();
-    private final List<List<LogicalVariable>> correlatedNullableVariableLists = new ArrayList<List<LogicalVariable>>();
-    private final ILogicalOperator op;
-    private final ITypingContext ctx;
-
-    public PropagateOperatorInputsTypeEnvironment(ILogicalOperator op, ITypingContext ctx,
-            IExpressionTypeComputer expressionTypeComputer, IMetadataProvider<?, ?> metadataProvider) {
-        super(expressionTypeComputer, metadataProvider);
-        this.op = op;
-        this.ctx = ctx;
-    }
-
-    public List<LogicalVariable> getNonNullVariables() {
-        return nonNullVariables;
-    }
-
-    @Override
-    public Object getVarType(LogicalVariable var, List<LogicalVariable> nonNullVariableList,
-            List<List<LogicalVariable>> correlatedNullableVariableLists) throws AlgebricksException {
-        nonNullVariableList.addAll(nonNullVariables);
-        return getVarTypeFullList(var, nonNullVariableList, correlatedNullableVariableLists);
-    }
-
-    private Object getVarTypeFullList(LogicalVariable var, List<LogicalVariable> nonNullVariableList,
-            List<List<LogicalVariable>> correlatedNullableVariableLists) throws AlgebricksException {
-        Object t = varTypeMap.get(var);
-        if (t != null) {
-            return t;
-        }
-        for (Mutable<ILogicalOperator> r : op.getInputs()) {
-            ILogicalOperator c = r.getValue();
-            IVariableTypeEnvironment env = ctx.getOutputTypeEnvironment(c);
-            Object t2 = env.getVarType(var, nonNullVariableList, correlatedNullableVariableLists);
-            if (t2 != null) {
-                return t2;
-            }
-        }
-        return null;
-    }
-
-    @Override
-    public Object getVarType(LogicalVariable var) throws AlgebricksException {
-        return getVarTypeFullList(var, nonNullVariables, correlatedNullableVariableLists);
-    }
-
-}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/typing/PropagatingTypeEnvironment.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/typing/PropagatingTypeEnvironment.java
index 27aa902..96b55d5 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/typing/PropagatingTypeEnvironment.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/typing/PropagatingTypeEnvironment.java
@@ -27,6 +27,7 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.IMissableTypeComputer;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
 import org.apache.hyracks.algebricks.core.algebra.properties.TypePropagationPolicy;
+import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
 
 public class PropagatingTypeEnvironment extends AbstractTypeEnvironment {
 
@@ -40,6 +41,10 @@
 
     private final List<List<LogicalVariable>> correlatedMissableVariableLists = new ArrayList<>();
 
+    private final List<LogicalVariable> nonNullableVariables = new ArrayList<>();
+
+    private final List<List<LogicalVariable>> correlatedNullableVariableLists = new ArrayList<>();
+
     public PropagatingTypeEnvironment(IExpressionTypeComputer expressionTypeComputer,
             IMissableTypeComputer missableTypeComputer, IMetadataProvider<?, ?> metadataProvider,
             TypePropagationPolicy policy, ITypeEnvPointer[] envPointers) {
@@ -51,7 +56,7 @@
 
     @Override
     public Object getVarType(LogicalVariable var) throws AlgebricksException {
-        return getVarTypeFullList(var, nonMissableVariables, correlatedMissableVariableLists);
+        return getVarTypeFullList(var, null, null, null, null);
     }
 
     public List<LogicalVariable> getNonMissableVariables() {
@@ -62,31 +67,42 @@
         return correlatedMissableVariableLists;
     }
 
-    @Override
-    public Object getVarType(LogicalVariable var, List<LogicalVariable> nonMissableVariableList,
-            List<List<LogicalVariable>> correlatedMissableVariableLists) throws AlgebricksException {
-        for (LogicalVariable v : nonMissableVariables) {
-            if (!nonMissableVariableList.contains(v)) {
-                nonMissableVariableList.add(v);
-            }
-        }
-        Object t = getVarTypeFullList(var, nonMissableVariableList, correlatedMissableVariableLists);
-        for (List<LogicalVariable> list : correlatedMissableVariableLists) {
-            if (!correlatedMissableVariableLists.contains(list)) {
-                correlatedMissableVariableLists.add(list);
-            }
-        }
-        return t;
+    public List<LogicalVariable> getNonNullableVariables() {
+        return nonNullableVariables;
     }
 
-    private Object getVarTypeFullList(LogicalVariable var, List<LogicalVariable> nonMissableVariableList,
-            List<List<LogicalVariable>> correlatedMissableVariableLists) throws AlgebricksException {
+    public List<List<LogicalVariable>> getCorrelatedNullableVariableLists() {
+        return correlatedNullableVariableLists;
+    }
+
+    @Override
+    public Object getVarType(LogicalVariable var, List<LogicalVariable> nonMissableVariables,
+            List<List<LogicalVariable>> correlatedMissableVariableLists, List<LogicalVariable> nonNullableVariables,
+            List<List<LogicalVariable>> correlatedNullableVariableLists) throws AlgebricksException {
+        return getVarTypeFullList(var, nonMissableVariables, correlatedMissableVariableLists, nonNullableVariables,
+                correlatedNullableVariableLists);
+    }
+
+    private Object getVarTypeFullList(LogicalVariable var, List<LogicalVariable> nonMissableVariableListExtra,
+            List<List<LogicalVariable>> correlatedMissableVariableListsExtra,
+            List<LogicalVariable> nonNullableVariableListExtra,
+            List<List<LogicalVariable>> correlatedNullableVariableListsExtra) throws AlgebricksException {
         Object t = varTypeMap.get(var);
         if (t != null) {
             return t;
         }
-        return policy.getVarType(var, missableTypeComputer, nonMissableVariableList, correlatedMissableVariableLists,
-                envPointers);
+
+        List<LogicalVariable> nonMissable =
+                OperatorPropertiesUtil.unionAll(nonMissableVariables, nonMissableVariableListExtra);
+        List<LogicalVariable> nonNullable =
+                OperatorPropertiesUtil.unionAll(nonNullableVariables, nonNullableVariableListExtra);
+        List<List<LogicalVariable>> correlatedMissable =
+                OperatorPropertiesUtil.unionAll(correlatedMissableVariableLists, correlatedMissableVariableListsExtra);
+        List<List<LogicalVariable>> correlatedNullable =
+                OperatorPropertiesUtil.unionAll(correlatedNullableVariableLists, correlatedNullableVariableListsExtra);
+
+        return policy.getVarType(var, missableTypeComputer, nonMissable, correlatedMissable, nonNullable,
+                correlatedNullable, envPointers);
     }
 
     @Override
@@ -100,6 +116,14 @@
                 missableVarList.add(v2);
             }
         }
+        if (nonNullableVariables.remove(v1)) {
+            nonNullableVariables.add(v2);
+        }
+        for (List<LogicalVariable> nullableVarList : correlatedNullableVariableLists) {
+            if (nullableVarList.remove(v1)) {
+                nullableVarList.add(v2);
+            }
+        }
         return result;
     }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorPropertiesUtil.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorPropertiesUtil.java
index c0d72bb..c66ee40 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorPropertiesUtil.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorPropertiesUtil.java
@@ -18,8 +18,11 @@
  */
 package org.apache.hyracks.algebricks.core.algebra.util;
 
+import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Set;
 
 import org.apache.commons.lang3.mutable.Mutable;
@@ -32,13 +35,13 @@
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
-import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
 import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.CardinalityInferenceVisitor;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
 import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
@@ -221,33 +224,6 @@
         }
     }
 
-    public static boolean isMissingTest(AbstractLogicalOperator op) {
-        if (op.getOperatorTag() != LogicalOperatorTag.SELECT) {
-            return false;
-        }
-        AbstractLogicalOperator doubleUnder = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
-        if (doubleUnder.getOperatorTag() != LogicalOperatorTag.NESTEDTUPLESOURCE) {
-            return false;
-        }
-        ILogicalExpression eu = ((SelectOperator) op).getCondition().getValue();
-        if (eu.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
-            return false;
-        }
-        AbstractFunctionCallExpression f1 = (AbstractFunctionCallExpression) eu;
-        if (!f1.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.NOT)) {
-            return false;
-        }
-        ILogicalExpression a1 = f1.getArguments().get(0).getValue();
-        if (!a1.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)) {
-            return false;
-        }
-        AbstractFunctionCallExpression f2 = (AbstractFunctionCallExpression) a1;
-        if (!f2.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.IS_MISSING)) {
-            return false;
-        }
-        return true;
-    }
-
     public static void typePlan(ILogicalPlan p, IOptimizationContext context) throws AlgebricksException {
         for (Mutable<ILogicalOperator> r : p.getRoots()) {
             typeOpRec(r, context);
@@ -355,4 +331,22 @@
         LogicalOperatorTag opTag = op.getOperatorTag();
         return opTag == LogicalOperatorTag.REPLICATE || opTag == LogicalOperatorTag.SPLIT;
     }
+
+    public static <T> List<T> unionAll(List<T> list1, List<T> list2) {
+        if (list2 == null || list2.isEmpty()) {
+            return list1 == null ? Collections.emptyList() : list1;
+        } else if (list1 == null || list1.isEmpty()) {
+            return list2;
+        } else {
+            List<T> result = new ArrayList<>(list1.size() + list2.size());
+            result.addAll(list1);
+            result.addAll(list2);
+            return result;
+        }
+    }
+
+    public static FunctionIdentifier getIsMissingNullFunction(IAlgebricksConstantValue value) {
+        return value.isMissing() ? AlgebricksBuiltinFunctions.IS_MISSING
+                : value.isNull() ? AlgebricksBuiltinFunctions.IS_NULL : null;
+    }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenContext.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenContext.java
index 2ca4a36..7c7d5a8 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenContext.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenContext.java
@@ -60,6 +60,7 @@
     private final ITypeTraitProvider typeTraitProvider;
     private final IMetadataProvider<?, ?> metadataProvider;
     private final IMissingWriterFactory missingWriterFactory;
+    private final IMissingWriterFactory nullWriterFactory;
     private final IUnnestingPositionWriterFactory unnestingPositionWriterFactory;
     private final INormalizedKeyComputerFactoryProvider normalizedKeyComputerFactoryProvider;
     private final Object appContext;
@@ -85,7 +86,8 @@
             IBinaryComparatorFactoryProvider comparatorFactoryProvider, ITypeTraitProvider typeTraitProvider,
             IBinaryBooleanInspectorFactory booleanInspectorFactory,
             IBinaryIntegerInspectorFactory integerInspectorFactory, IPrinterFactoryProvider printerFactoryProvider,
-            IMissingWriterFactory missingWriterFactory, IUnnestingPositionWriterFactory unnestingPositionWriterFactory,
+            IMissingWriterFactory missingWriterFactory, IMissingWriterFactory nullWriterFactory,
+            IUnnestingPositionWriterFactory unnestingPositionWriterFactory,
             INormalizedKeyComputerFactoryProvider normalizedKeyComputerFactoryProvider,
             IExpressionRuntimeProvider expressionRuntimeProvider, IExpressionTypeComputer expressionTypeComputer,
             ITypingContext typingContext, IExpressionEvalSizeComputer expressionEvalSizeComputer,
@@ -107,6 +109,7 @@
         this.clusterLocations = clusterLocations;
         this.normalizedKeyComputerFactoryProvider = normalizedKeyComputerFactoryProvider;
         this.missingWriterFactory = missingWriterFactory;
+        this.nullWriterFactory = nullWriterFactory;
         this.unnestingPositionWriterFactory = unnestingPositionWriterFactory;
         this.expressionRuntimeProvider = expressionRuntimeProvider;
         this.expressionTypeComputer = expressionTypeComputer;
@@ -199,6 +202,10 @@
         return missingWriterFactory;
     }
 
+    public IMissingWriterFactory getNullWriterFactory() {
+        return nullWriterFactory;
+    }
+
     public IUnnestingPositionWriterFactory getUnnestingPositionWriterFactory() {
         return unnestingPositionWriterFactory;
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenHelper.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenHelper.java
index 5142ce7..42364c8 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenHelper.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenHelper.java
@@ -18,11 +18,13 @@
  */
 package org.apache.hyracks.algebricks.core.jobgen.impl;
 
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
 
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
@@ -38,10 +40,12 @@
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import org.apache.hyracks.api.dataflow.value.IBinaryHashFunctionFamily;
+import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
 import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -213,4 +217,25 @@
         }
         return projectionList;
     }
+
+    public static IMissingWriterFactory[] createMissingWriterFactories(JobGenContext context,
+            IAlgebricksConstantValue missingOrNullValue, int size) throws AlgebricksException {
+        IMissingWriterFactory missingWriterFactory = getMissingWriterFactory(context, missingOrNullValue);
+        IMissingWriterFactory[] nonMatchWriterFactories = new IMissingWriterFactory[size];
+        Arrays.fill(nonMatchWriterFactories, missingWriterFactory);
+        return nonMatchWriterFactories;
+    }
+
+    public static IMissingWriterFactory getMissingWriterFactory(JobGenContext context,
+            IAlgebricksConstantValue missingOrNullValue) throws AlgebricksException {
+        IMissingWriterFactory missingWriterFactory;
+        if (missingOrNullValue.isMissing()) {
+            missingWriterFactory = context.getMissingWriterFactory();
+        } else if (missingOrNullValue.isNull()) {
+            missingWriterFactory = context.getNullWriterFactory();
+        } else {
+            throw new AlgebricksException(ErrorCode.ILLEGAL_STATE);
+        }
+        return missingWriterFactory;
+    }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java
index 01f7403..ee7f7aa 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java
@@ -166,7 +166,9 @@
     @Override
     public String visitLeftOuterJoinOperator(LeftOuterJoinOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
-        stringBuilder.append("left outer join (").append(op.getCondition().getValue().toString()).append(")");
+        stringBuilder.append("left outer join ").append("(").append(op.getCondition().getValue().toString())
+                .append(")");
+        stringBuilder.append(op.getMissingValue().isNull() ? " (or null) " : "");
         appendSchema(op, showDetails);
         appendAnnotations(op, showDetails);
         appendPhysicalOperatorInfo(op, showDetails);
@@ -351,6 +353,7 @@
         if (op.getPositionalVariable() != null) {
             stringBuilder.append(" at ").append(op.getPositionalVariable());
         }
+        stringBuilder.append(op.getMissingValue().isNull() ? " (or null) " : "");
         stringBuilder.append(" <- ").append(op.getExpressionRef().getValue().toString());
         appendSchema(op, showDetails);
         appendAnnotations(op, showDetails);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/test/java/org/apache/hyracks/algebricks/core/algebra/plan/PlanStructureVerifierTest.java b/hyracks-fullstack/algebricks/algebricks-core/src/test/java/org/apache/hyracks/algebricks/core/algebra/plan/PlanStructureVerifierTest.java
index f047cec..46e41f1 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/test/java/org/apache/hyracks/algebricks/core/algebra/plan/PlanStructureVerifierTest.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/test/java/org/apache/hyracks/algebricks/core/algebra/plan/PlanStructureVerifierTest.java
@@ -283,14 +283,14 @@
         computeAndSetTypeEnvironmentForOperator(ets);
         ets.recomputeSchema();
 
-        SelectOperator op1 = new SelectOperator(newMutable(ConstantExpression.TRUE), false, null);
+        SelectOperator op1 = new SelectOperator(newMutable(ConstantExpression.TRUE));
         op1.getInputs().add(newMutable(ets));
         computeAndSetTypeEnvironmentForOperator(op1);
         op1.recomputeSchema();
 
         op1.getInputs().clear();
 
-        SelectOperator op2 = new SelectOperator(newMutable(ConstantExpression.FALSE), false, null);
+        SelectOperator op2 = new SelectOperator(newMutable(ConstantExpression.FALSE));
         op2.getInputs().add(newMutable(ets));
         op2.recomputeSchema();
         // no type env
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/BreakSelectIntoConjunctsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/BreakSelectIntoConjunctsRule.java
index 50c7703..c0773b2 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/BreakSelectIntoConjunctsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/BreakSelectIntoConjunctsRule.java
@@ -105,9 +105,9 @@
                 firstExpression = expression;
             } else {
                 // New select operator
-                SelectOperator newSelectOperator =
-                        new SelectOperator(new MutableObject<>(expression), originalSelectOperator.getRetainMissing(),
-                                originalSelectOperator.getMissingPlaceholderVariable());
+                SelectOperator newSelectOperator = new SelectOperator(new MutableObject<>(expression),
+                        originalSelectOperator.getRetainMissingAsValue(),
+                        originalSelectOperator.getMissingPlaceholderVariable());
                 newSelectOperator.setSourceLocation(sourceLoc);
 
                 // Put the new operator at the bottom (child of current operator)
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PullSelectOutOfEqJoin.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PullSelectOutOfEqJoin.java
index afbbc4f..410b2ec 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PullSelectOutOfEqJoin.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PullSelectOutOfEqJoin.java
@@ -80,7 +80,7 @@
         }
         // pull up
         ILogicalExpression pulledCond = makeCondition(otherPredicates, context);
-        SelectOperator select = new SelectOperator(new MutableObject<ILogicalExpression>(pulledCond), false, null);
+        SelectOperator select = new SelectOperator(new MutableObject<ILogicalExpression>(pulledCond));
         ILogicalExpression newJoinCond = makeCondition(eqVarVarComps, context);
         join.getCondition().setValue(newJoinCond);
         select.getInputs().add(new MutableObject<ILogicalOperator>(join));
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushSelectIntoJoinRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushSelectIntoJoinRule.java
index 5f66c2f..5477662 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushSelectIntoJoinRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushSelectIntoJoinRule.java
@@ -38,9 +38,11 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
@@ -98,10 +100,10 @@
             ILogicalOperator joinBranchRight = joinBranchRightRef.getValue();
             VariableUtilities.getLiveVariables(joinBranchLeft, joinLiveVarsLeft);
             VariableUtilities.getLiveVariables(joinBranchRight, joinLiveVarsRight);
-            Mutable<ILogicalOperator> opIterRef = opRef2;
+            Mutable<ILogicalOperator> opIterRef;
             ILogicalOperator opIter = op2;
             while (opIter != join) {
-                LogicalOperatorTag tag = ((AbstractLogicalOperator) opIter).getOperatorTag();
+                LogicalOperatorTag tag = opIter.getOperatorTag();
                 if (tag == LogicalOperatorTag.PROJECT) {
                     notPushedStack.addFirst(opIter);
                 } else {
@@ -129,11 +131,11 @@
 
         boolean intersectsAllBranches = true;
         boolean[] intersectsBranch = new boolean[join.getInputs().size()];
-        LinkedList<LogicalVariable> selectVars = new LinkedList<LogicalVariable>();
+        LinkedList<LogicalVariable> selectVars = new LinkedList<>();
         select.getCondition().getValue().getUsedVariables(selectVars);
         int i = 0;
         for (Mutable<ILogicalOperator> branch : join.getInputs()) {
-            LinkedList<LogicalVariable> branchVars = new LinkedList<LogicalVariable>();
+            LinkedList<LogicalVariable> branchVars = new LinkedList<>();
             VariableUtilities.getLiveVariables(branch.getValue(), branchVars);
             if (i == 0) {
                 branchVars.addAll(liveInOpsToPushLeft);
@@ -181,7 +183,9 @@
                 if (j > 0 && isLoj) {
                     // if a LOJ and the select condition is not-missing filtering,
                     // we rewrite LOJ to IJ for this case.
-                    if (containsNotMissingFiltering(selectCondition)) {
+                    FunctionIdentifier isMissingNullFunction = OperatorPropertiesUtil
+                            .getIsMissingNullFunction(((LeftOuterJoinOperator) join).getMissingValue());
+                    if (containsNotMissingFiltering(selectCondition, isMissingNullFunction)) {
                         lojToInner = true;
                     }
                     // Do not push conditions into the right branch of a LOJ;
@@ -247,7 +251,7 @@
             ILogicalOperator op = iter.previous();
             List<Mutable<ILogicalOperator>> opInpList = op.getInputs();
             opInpList.clear();
-            opInpList.add(new MutableObject<ILogicalOperator>(topOp));
+            opInpList.add(new MutableObject<>(topOp));
             topOp = op;
             context.computeAndSetTypeEnvironmentForOperator(op);
         }
@@ -276,7 +280,7 @@
             if (!bAddedToConj) {
                 AbstractFunctionCallExpression newCond = new ScalarFunctionCallExpression(
                         context.getMetadataProvider().lookupFunction(AlgebricksBuiltinFunctions.AND),
-                        select.getCondition(), new MutableObject<ILogicalExpression>(join.getCondition().getValue()));
+                        select.getCondition(), new MutableObject<>(join.getCondition().getValue()));
                 join.getCondition().setValue(newCond);
             }
         }
@@ -284,9 +288,9 @@
 
     private static void copySelectToBranch(SelectOperator select, Mutable<ILogicalOperator> branch,
             IOptimizationContext context) throws AlgebricksException {
-        ILogicalOperator newSelect = new SelectOperator(select.getCondition(), select.getRetainMissing(),
+        ILogicalOperator newSelect = new SelectOperator(select.getCondition(), select.getRetainMissingAsValue(),
                 select.getMissingPlaceholderVariable());
-        Mutable<ILogicalOperator> newRef = new MutableObject<ILogicalOperator>(branch.getValue());
+        Mutable<ILogicalOperator> newRef = new MutableObject<>(branch.getValue());
         newSelect.getInputs().add(newRef);
         branch.setValue(newSelect);
         context.computeAndSetTypeEnvironmentForOperator(newSelect);
@@ -296,16 +300,17 @@
      * Whether the expression contains a not-missing filtering
      *
      * @param expr
+     * @param isMissingNullFunId
      * @return true if the expression contains a not-missing filtering function call; false otherwise.
      */
-    private boolean containsNotMissingFiltering(ILogicalExpression expr) {
+    private boolean containsNotMissingFiltering(ILogicalExpression expr, FunctionIdentifier isMissingNullFunId) {
         if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
             return false;
         }
         ScalarFunctionCallExpression func = (ScalarFunctionCallExpression) expr;
         if (func.getFunctionIdentifier() == AlgebricksBuiltinFunctions.AND) {
             for (Mutable<ILogicalExpression> argumentRef : func.getArguments()) {
-                if (containsNotMissingFiltering(argumentRef.getValue())) {
+                if (containsNotMissingFiltering(argumentRef.getValue(), isMissingNullFunId)) {
                     return true;
                 }
             }
@@ -319,34 +324,6 @@
             return false;
         }
         ScalarFunctionCallExpression func2 = (ScalarFunctionCallExpression) arg;
-        if (func2.getFunctionIdentifier() != AlgebricksBuiltinFunctions.IS_MISSING) {
-            return false;
-        }
-        return true;
-    }
-
-    /**
-     * Whether the expression contains a missing filtering
-     *
-     * @param expr
-     * @return true if the expression contains a missing filtering function call; false otherwise.
-     */
-    private boolean containsMissingFiltering(ILogicalExpression expr) {
-        if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
-            return false;
-        }
-        ScalarFunctionCallExpression func = (ScalarFunctionCallExpression) expr;
-        if (func.getFunctionIdentifier() == AlgebricksBuiltinFunctions.AND) {
-            for (Mutable<ILogicalExpression> argumentRef : func.getArguments()) {
-                if (containsMissingFiltering(argumentRef.getValue())) {
-                    return true;
-                }
-            }
-            return false;
-        }
-        if (func.getFunctionIdentifier() != AlgebricksBuiltinFunctions.IS_MISSING) {
-            return false;
-        }
-        return true;
+        return func2.getFunctionIdentifier().equals(isMissingNullFunId);
     }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceGroupByForSubplanRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceGroupByForSubplanRule.java
index c1e613b..391d03a 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceGroupByForSubplanRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceGroupByForSubplanRule.java
@@ -65,6 +65,8 @@
 import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
+ * NOTE: This rule is currently not used
+ *
  * The rule searches for SUBPLAN operator with a optional PROJECT operator and
  * an AGGREGATE followed by a join operator.
  *
@@ -258,8 +260,7 @@
         ScalarFunctionCallExpression nonNullTest =
                 new ScalarFunctionCallExpression(finfoNot, new MutableObject<ILogicalExpression>(isNullTest));
         nonNullTest.setSourceLocation(sourceLoc);
-        SelectOperator selectNonNull =
-                new SelectOperator(new MutableObject<ILogicalExpression>(nonNullTest), false, null);
+        SelectOperator selectNonNull = new SelectOperator(new MutableObject<ILogicalExpression>(nonNullTest));
         selectNonNull.setSourceLocation(sourceLoc);
         GroupByOperator g = new GroupByOperator();
         g.setSourceLocation(sourceLoc);
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceLeftOuterJoinForSubplanRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceLeftOuterJoinForSubplanRule.java
index e37be1b..d7f8d38 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceLeftOuterJoinForSubplanRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/IntroduceLeftOuterJoinForSubplanRule.java
@@ -22,17 +22,26 @@
 
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
 import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
-import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 
+/**
+ * NOTE: This rule is currently not used
+ *
+ */
 public class IntroduceLeftOuterJoinForSubplanRule implements IAlgebraicRewriteRule {
 
     @Override
@@ -65,7 +74,7 @@
         AbstractLogicalOperator op1 = (AbstractLogicalOperator) subplanRoot.getValue();
         Mutable<ILogicalOperator> opUnder = subplan.getInputs().get(0);
 
-        if (OperatorPropertiesUtil.isMissingTest((AbstractLogicalOperator) opUnder.getValue())) {
+        if (isMissingTest((AbstractLogicalOperator) opUnder.getValue())) {
             return false;
         }
 
@@ -87,7 +96,8 @@
                     }
                 }
                 ntsRef.setValue(opUnder.getValue());
-                LeftOuterJoinOperator loj = new LeftOuterJoinOperator(join.getCondition());
+                LeftOuterJoinOperator loj =
+                        new LeftOuterJoinOperator(join.getCondition(), ConstantExpression.MISSING.getValue());
                 loj.setSourceLocation(join.getSourceLocation());
                 loj.getInputs().add(leftRef);
                 loj.getInputs().add(rightRef);
@@ -124,4 +134,30 @@
         return getNtsAtEndOfPipeline(op.getInputs().get(0));
     }
 
+    public static boolean isMissingTest(AbstractLogicalOperator op) {
+        if (op.getOperatorTag() != LogicalOperatorTag.SELECT) {
+            return false;
+        }
+        AbstractLogicalOperator doubleUnder = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
+        if (doubleUnder.getOperatorTag() != LogicalOperatorTag.NESTEDTUPLESOURCE) {
+            return false;
+        }
+        ILogicalExpression eu = ((SelectOperator) op).getCondition().getValue();
+        if (eu.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            return false;
+        }
+        AbstractFunctionCallExpression f1 = (AbstractFunctionCallExpression) eu;
+        if (!f1.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.NOT)) {
+            return false;
+        }
+        ILogicalExpression a1 = f1.getArguments().get(0).getValue();
+        if (!a1.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)) {
+            return false;
+        }
+        AbstractFunctionCallExpression f2 = (AbstractFunctionCallExpression) a1;
+        if (!f2.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.IS_MISSING)) {
+            return false;
+        }
+        return true;
+    }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/SubplanOutOfGroupRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/SubplanOutOfGroupRule.java
index d3ecf69..428b710 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/SubplanOutOfGroupRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/SubplanOutOfGroupRule.java
@@ -25,19 +25,25 @@
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
 import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
-import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 
 /**
+ * NOTE: This rule is currently not used
+ *
  * Looks for a nested group-by plan ending in
  * subplan {
  * ...
@@ -82,7 +88,7 @@
             if (op1.getOperatorTag() == LogicalOperatorTag.SUBPLAN) {
                 SubplanOperator subplan = (SubplanOperator) op1;
                 AbstractLogicalOperator op2 = (AbstractLogicalOperator) subplan.getInputs().get(0).getValue();
-                if (OperatorPropertiesUtil.isMissingTest(op2)) {
+                if (isMissingTest(op2)) {
                     if (subplan.getNestedPlans().size() == 1) {
                         ILogicalPlan p1 = subplan.getNestedPlans().get(0);
                         if (p1.getRoots().size() == 1) {
@@ -122,4 +128,31 @@
 
         return true;
     }
+
+    private static boolean isMissingTest(AbstractLogicalOperator op) {
+        if (op.getOperatorTag() != LogicalOperatorTag.SELECT) {
+            return false;
+        }
+        AbstractLogicalOperator doubleUnder = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
+        if (doubleUnder.getOperatorTag() != LogicalOperatorTag.NESTEDTUPLESOURCE) {
+            return false;
+        }
+        ILogicalExpression eu = ((SelectOperator) op).getCondition().getValue();
+        if (eu.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            return false;
+        }
+        AbstractFunctionCallExpression f1 = (AbstractFunctionCallExpression) eu;
+        if (!f1.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.NOT)) {
+            return false;
+        }
+        ILogicalExpression a1 = f1.getArguments().get(0).getValue();
+        if (!a1.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)) {
+            return false;
+        }
+        AbstractFunctionCallExpression f2 = (AbstractFunctionCallExpression) a1;
+        if (!f2.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.IS_MISSING)) {
+            return false;
+        }
+        return true;
+    }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/UnnestRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/UnnestRuntimeFactory.java
index 9ee5a48..1b42bb6 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/UnnestRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/UnnestRuntimeFactory.java
@@ -18,8 +18,6 @@
  */
 package org.apache.hyracks.algebricks.runtime.operators.std;
 
-import java.io.ByteArrayOutputStream;
-import java.io.DataOutput;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
@@ -38,6 +36,7 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 
 public class UnnestRuntimeFactory extends AbstractOneInputOneOutputRuntimeFactory {
@@ -83,14 +82,7 @@
     @Override
     public AbstractOneInputOneOutputOneFramePushRuntime createOneOutputPushRuntime(final IHyracksTaskContext ctx)
             throws HyracksDataException {
-        ByteArrayOutputStream bos = new ByteArrayOutputStream();
-        DataOutput output = new DataOutputStream(bos);
-        if (missingWriterFactory != null) {
-            IMissingWriter missingWriter = missingWriterFactory.createMissingWriter();
-            missingWriter.writeMissing(output);
-        }
-        byte[] missingBytes = bos.toByteArray();
-        int missingBytesLen = bos.size();
+        ByteArrayAccessibleOutputStream missingBytes = leftOuter ? writeMissingBytes() : null;
         IEvaluatorContext evalCtx = new EvaluatorContext(ctx);
         return new AbstractOneInputOneOutputOneFramePushRuntime() {
             private IPointable p = VoidPointable.FACTORY.createPointable();
@@ -154,7 +146,7 @@
                 }
                 if (unnestColIsProjected) {
                     if (missing) {
-                        tupleBuilder.addField(missingBytes, 0, missingBytesLen);
+                        tupleBuilder.addField(missingBytes.getByteArray(), 0, missingBytes.size());
                     } else {
                         tupleBuilder.addField(p.getByteArray(), p.getStartOffset(), p.getLength());
                     }
@@ -166,7 +158,7 @@
                 if (positionWriter != null) {
                     // Write the positional variable
                     if (missing) {
-                        tupleBuilder.addField(missingBytes, 0, missingBytesLen);
+                        tupleBuilder.addField(missingBytes.getByteArray(), 0, missingBytes.size());
                     } else {
                         positionWriter.write(tupleBuilder.getDataOutput(), positionIndex);
                         tupleBuilder.addFieldEndOffset();
@@ -181,4 +173,11 @@
             }
         };
     }
+
+    private ByteArrayAccessibleOutputStream writeMissingBytes() throws HyracksDataException {
+        ByteArrayAccessibleOutputStream baos = new ByteArrayAccessibleOutputStream();
+        IMissingWriter missingWriter = missingWriterFactory.createMissingWriter();
+        missingWriter.writeMissing(new DataOutputStream(baos));
+        return baos;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/PrimaryIndexSearchExample.java b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
index ab3db7d..20e8eae 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
@@ -143,7 +143,7 @@
                 new IndexDataflowHelperFactory(storageManager, btreeSplitProvider);
         BTreeSearchOperatorDescriptor btreeSearchOp = new BTreeSearchOperatorDescriptor(spec, recDesc, lowKeyFields,
                 highKeyFields, true, true, dataflowHelperFactory, false, false, null,
-                NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+                NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
 
         JobHelper.createPartitionConstraint(spec, btreeSearchOp, splitNCs);
 
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/SecondaryIndexSearchExample.java b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
index 44b982b..26ddbd6 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/src/main/java/org/apache/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
@@ -170,7 +170,7 @@
                 new IndexDataflowHelperFactory(storageManager, secondarySplitProvider);
         BTreeSearchOperatorDescriptor secondarySearchOp = new BTreeSearchOperatorDescriptor(spec, secondaryRecDesc,
                 secondaryLowKeyFields, secondaryHighKeyFields, true, true, secondaryHelperFactory, false, false, null,
-                NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+                NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
 
         JobHelper.createPartitionConstraint(spec, secondarySearchOp, splitNCs);
 
@@ -188,7 +188,7 @@
                 new IndexDataflowHelperFactory(storageManager, primarySplitProvider);
         BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
                 primaryLowKeyFields, primaryHighKeyFields, true, true, primaryHelperFactory, false, false, null,
-                NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+                NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
 
         JobHelper.createPartitionConstraint(spec, primarySearchOp, splitNCs);
 
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/AbstractBTreeOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/AbstractBTreeOperatorTest.java
index 283b352..eac332d 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/AbstractBTreeOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/AbstractBTreeOperatorTest.java
@@ -185,7 +185,7 @@
         // scan primary index
         BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
                 lowKeyFields, highKeyFields, true, true, primaryHelperFactory, false, false, null,
-                NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+                NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
 
         // sort based on secondary keys
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreePrimaryIndexScanOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreePrimaryIndexScanOperatorTest.java
index e4214cc..9474b12 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreePrimaryIndexScanOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreePrimaryIndexScanOperatorTest.java
@@ -76,7 +76,7 @@
 
         BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
                 DataSetConstants.primaryRecDesc, lowKeyFields, highKeyFields, true, true, primaryHelperFactory, false,
-                false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+                false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
 
         IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreePrimaryIndexSearchOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreePrimaryIndexSearchOperatorTest.java
index 65bb2f0..af043ab 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreePrimaryIndexSearchOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreePrimaryIndexSearchOperatorTest.java
@@ -81,7 +81,7 @@
 
         BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
                 DataSetConstants.primaryRecDesc, lowKeyFields, highKeyFields, true, true, primaryHelperFactory, false,
-                false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+                false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
 
         IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexInsertOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexInsertOperatorTest.java
index b489474..93ab78d 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexInsertOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexInsertOperatorTest.java
@@ -83,9 +83,10 @@
         int[] secondaryHighKeyFields = { 1 };
 
         // search secondary index
-        BTreeSearchOperatorDescriptor secondaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
-                DataSetConstants.secondaryRecDesc, secondaryLowKeyFields, secondaryHighKeyFields, true, true,
-                secondaryHelperFactory, false, false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+        BTreeSearchOperatorDescriptor secondaryBtreeSearchOp =
+                new BTreeSearchOperatorDescriptor(spec, DataSetConstants.secondaryRecDesc, secondaryLowKeyFields,
+                        secondaryHighKeyFields, true, true, secondaryHelperFactory, false, false, null,
+                        NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
 
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeSearchOp, NC1_ID);
 
@@ -95,9 +96,10 @@
         int[] primaryHighKeyFields = { 1 };
 
         // search primary index
-        BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
-                DataSetConstants.primaryRecDesc, primaryLowKeyFields, primaryHighKeyFields, true, true,
-                primaryHelperFactory, false, false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+        BTreeSearchOperatorDescriptor primaryBtreeSearchOp =
+                new BTreeSearchOperatorDescriptor(spec, DataSetConstants.primaryRecDesc, primaryLowKeyFields,
+                        primaryHighKeyFields, true, true, primaryHelperFactory, false, false, null,
+                        NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
 
         IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexSearchOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexSearchOperatorTest.java
index a1d1502..a5a0bda 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexSearchOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexSearchOperatorTest.java
@@ -82,9 +82,10 @@
         int[] secondaryHighKeyFields = { 1 };
 
         // search secondary index
-        BTreeSearchOperatorDescriptor secondaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
-                DataSetConstants.secondaryRecDesc, secondaryLowKeyFields, secondaryHighKeyFields, true, true,
-                secondaryHelperFactory, false, false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+        BTreeSearchOperatorDescriptor secondaryBtreeSearchOp =
+                new BTreeSearchOperatorDescriptor(spec, DataSetConstants.secondaryRecDesc, secondaryLowKeyFields,
+                        secondaryHighKeyFields, true, true, secondaryHelperFactory, false, false, null,
+                        NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
 
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeSearchOp, NC1_ID);
 
@@ -94,9 +95,10 @@
         // coming from secondary index
 
         // search primary index
-        BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
-                DataSetConstants.primaryRecDesc, primaryLowKeyFields, primaryHighKeyFields, true, true,
-                primaryHelperFactory, false, false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+        BTreeSearchOperatorDescriptor primaryBtreeSearchOp =
+                new BTreeSearchOperatorDescriptor(spec, DataSetConstants.primaryRecDesc, primaryLowKeyFields,
+                        primaryHighKeyFields, true, true, primaryHelperFactory, false, false, null,
+                        NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
         IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
         IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexUpsertOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexUpsertOperatorTest.java
index f9cdfcf..3691cd4 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexUpsertOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/btree/BTreeSecondaryIndexUpsertOperatorTest.java
@@ -82,9 +82,10 @@
         int[] secondaryHighKeyFields = { 1 };
 
         // search secondary index
-        BTreeSearchOperatorDescriptor secondaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
-                DataSetConstants.secondaryRecDesc, secondaryLowKeyFields, secondaryHighKeyFields, true, true,
-                secondaryHelperFactory, false, false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+        BTreeSearchOperatorDescriptor secondaryBtreeSearchOp =
+                new BTreeSearchOperatorDescriptor(spec, DataSetConstants.secondaryRecDesc, secondaryLowKeyFields,
+                        secondaryHighKeyFields, true, true, secondaryHelperFactory, false, false, null,
+                        NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
 
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeSearchOp, NC1_ID);
 
@@ -94,9 +95,10 @@
         int[] primaryHighKeyFields = { 1 };
 
         // search primary index
-        BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
-                DataSetConstants.primaryRecDesc, primaryLowKeyFields, primaryHighKeyFields, true, true,
-                primaryHelperFactory, false, false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+        BTreeSearchOperatorDescriptor primaryBtreeSearchOp =
+                new BTreeSearchOperatorDescriptor(spec, DataSetConstants.primaryRecDesc, primaryLowKeyFields,
+                        primaryHighKeyFields, true, true, primaryHelperFactory, false, false, null,
+                        NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
 
         IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreePrimaryIndexSearchOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreePrimaryIndexSearchOperatorTest.java
index 9d3ba5c..da1441c 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreePrimaryIndexSearchOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreePrimaryIndexSearchOperatorTest.java
@@ -89,10 +89,10 @@
         int[] lowKeyFields = { 0 };
         int[] highKeyFields = { 1 };
 
-        BTreeSearchOperatorDescriptor primaryBtreeSearchOp =
-                new BTreeSearchOperatorDescriptor(spec, DataSetConstants.primaryAndFilterRecDesc, lowKeyFields,
-                        highKeyFields, true, true, primaryHelperFactory, false, false,
-                        NoopMissingWriterFactory.INSTANCE, NoOpOperationCallbackFactory.INSTANCE, null, null, true);
+        BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
+                DataSetConstants.primaryAndFilterRecDesc, lowKeyFields, highKeyFields, true, true, primaryHelperFactory,
+                false, false, NoopMissingWriterFactory.INSTANCE, NoOpOperationCallbackFactory.INSTANCE, null, null,
+                true, NoopMissingWriterFactory.INSTANCE);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
 
         IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
@@ -147,7 +147,7 @@
         BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
                 DataSetConstants.primaryAndFilterRecDesc, lowKeyFields, highKeyFields, true, true, primaryHelperFactory,
                 false, false, NoopMissingWriterFactory.INSTANCE, NoOpOperationCallbackFactory.INSTANCE, minFilterFields,
-                maxFilterFields, true);
+                maxFilterFields, true, NoopMissingWriterFactory.INSTANCE);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
 
         IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeSecondaryIndexSearchOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeSecondaryIndexSearchOperatorTest.java
index 97a4746..bed4794 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeSecondaryIndexSearchOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/btree/LSMBTreeSecondaryIndexSearchOperatorTest.java
@@ -90,10 +90,10 @@
         int[] secondaryHighKeyFields = { 1 };
 
         // search secondary index
-        BTreeSearchOperatorDescriptor secondaryBtreeSearchOp =
-                new BTreeSearchOperatorDescriptor(spec, DataSetConstants.secondaryWithFilterRecDesc,
-                        secondaryLowKeyFields, secondaryHighKeyFields, true, true, secondaryHelperFactory, false, false,
-                        NoopMissingWriterFactory.INSTANCE, NoOpOperationCallbackFactory.INSTANCE, null, null, true);
+        BTreeSearchOperatorDescriptor secondaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
+                DataSetConstants.secondaryWithFilterRecDesc, secondaryLowKeyFields, secondaryHighKeyFields, true, true,
+                secondaryHelperFactory, false, false, NoopMissingWriterFactory.INSTANCE,
+                NoOpOperationCallbackFactory.INSTANCE, null, null, true, NoopMissingWriterFactory.INSTANCE);
 
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeSearchOp, NC1_ID);
 
@@ -103,10 +103,11 @@
         int[] maxFilterFields = { 3 };
 
         // search primary index
-        BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
-                DataSetConstants.primaryRecDesc, primaryLowKeyFields, primaryHighKeyFields, true, true,
-                primaryHelperFactory, false, false, NoopMissingWriterFactory.INSTANCE,
-                NoOpOperationCallbackFactory.INSTANCE, minFilterFields, maxFilterFields, false);
+        BTreeSearchOperatorDescriptor primaryBtreeSearchOp =
+                new BTreeSearchOperatorDescriptor(spec, DataSetConstants.primaryRecDesc, primaryLowKeyFields,
+                        primaryHighKeyFields, true, true, primaryHelperFactory, false, false,
+                        NoopMissingWriterFactory.INSTANCE, NoOpOperationCallbackFactory.INSTANCE, minFilterFields,
+                        maxFilterFields, false, NoopMissingWriterFactory.INSTANCE);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
         IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
         IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesSecondaryIndexSearchOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesSecondaryIndexSearchOperatorTest.java
index 04179c2..6a31962 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesSecondaryIndexSearchOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesSecondaryIndexSearchOperatorTest.java
@@ -95,7 +95,7 @@
         int[] keyFields = { 0, 1, 2, 3 };
         RTreeSearchOperatorDescriptor secondarySearchOp = new RTreeSearchOperatorDescriptor(spec,
                 secondaryWithFilterRecDesc, keyFields, true, true, secondaryHelperFactory, false, false, null,
-                NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+                NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondarySearchOp, NC1_ID);
 
         IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/AbstractRTreeOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/AbstractRTreeOperatorTest.java
index 872502b..f849bc8 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/AbstractRTreeOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/AbstractRTreeOperatorTest.java
@@ -320,7 +320,7 @@
         // scan primary index
         BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
                 lowKeyFields, highKeyFields, true, true, primaryHelperFactory, false, false, null,
-                NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+                NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primarySearchOp, NC1_ID);
 
         // load secondary index
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexInsertOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexInsertOperatorTest.java
index 784cd10..0fcf892 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexInsertOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexInsertOperatorTest.java
@@ -88,7 +88,7 @@
         int[] keyFields = { 0, 1, 2, 3 };
         RTreeSearchOperatorDescriptor secondarySearchOp =
                 new RTreeSearchOperatorDescriptor(spec, secondaryRecDesc, keyFields, true, true, secondaryHelperFactory,
-                        false, false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+                        false, false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondarySearchOp, NC1_ID);
         // fifth field from the tuples coming from secondary index
         int[] primaryLowKeyFields = { 4 };
@@ -97,7 +97,7 @@
         // search primary index
         BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
                 primaryLowKeyFields, primaryHighKeyFields, true, true, primaryHelperFactory, false, false, null,
-                NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+                NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primarySearchOp, NC1_ID);
         IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
         IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexScanOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexScanOperatorTest.java
index ff44d10..3cfef19 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexScanOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexScanOperatorTest.java
@@ -86,7 +86,7 @@
         int[] keyFields = null;
         RTreeSearchOperatorDescriptor secondarySearchOp =
                 new RTreeSearchOperatorDescriptor(spec, secondaryRecDesc, keyFields, true, true, secondaryHelperFactory,
-                        false, false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+                        false, false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondarySearchOp, NC1_ID);
         IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
         IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexSearchOperatorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexSearchOperatorTest.java
index e234322..8ea0701 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexSearchOperatorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/am/rtree/RTreeSecondaryIndexSearchOperatorTest.java
@@ -86,7 +86,7 @@
         int[] keyFields = { 0, 1, 2, 3 };
         RTreeSearchOperatorDescriptor secondarySearchOp =
                 new RTreeSearchOperatorDescriptor(spec, secondaryRecDesc, keyFields, true, true, secondaryHelperFactory,
-                        false, false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+                        false, false, null, NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondarySearchOp, NC1_ID);
         // fifth field from the tuples coming from secondary index
         int[] primaryLowKeyFields = { 4 };
@@ -95,7 +95,7 @@
         // search primary index
         BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
                 primaryLowKeyFields, primaryHighKeyFields, true, true, primaryHelperFactory, false, false, null,
-                NoOpOperationCallbackFactory.INSTANCE, null, null, false);
+                NoOpOperationCallbackFactory.INSTANCE, null, null, false, null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primarySearchOp, NC1_ID);
         IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { createFile(nc1) });
         IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
index 128929c..2455625 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
@@ -46,6 +46,7 @@
     protected final IMissingWriterFactory missingWriterFactory;
     protected final ISearchOperationCallbackFactory searchCallbackFactory;
     protected final boolean appendIndexFilter;
+    protected final IMissingWriterFactory nonFilterWriterFactory;
     protected boolean appendOpCallbackProceedResult;
     protected byte[] searchCallbackProceedResultFalseValue;
     protected byte[] searchCallbackProceedResultTrueValue;
@@ -56,10 +57,11 @@
             int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
             IIndexDataflowHelperFactory indexHelperFactory, boolean retainInput, boolean retainMissing,
             IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
-            int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, boolean appendIndexFilter) {
+            int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, boolean appendIndexFilter,
+            IMissingWriterFactory nonFilterWriterFactory) {
         this(spec, outRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory,
                 retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, minFilterFieldIndexes,
-                maxFilterFieldIndexes, appendIndexFilter, null, -1, false, null, null);
+                maxFilterFieldIndexes, appendIndexFilter, nonFilterWriterFactory, null, -1, false, null, null);
     }
 
     public BTreeSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor outRecDesc,
@@ -67,8 +69,9 @@
             IIndexDataflowHelperFactory indexHelperFactory, boolean retainInput, boolean retainMissing,
             IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
             int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, boolean appendIndexFilter,
-            ITupleFilterFactory tupleFilterFactory, long outputLimit, boolean appendOpCallbackProceedResult,
-            byte[] searchCallbackProceedResultFalseValue, byte[] searchCallbackProceedResultTrueValue) {
+            IMissingWriterFactory nonFilterWriterFactory, ITupleFilterFactory tupleFilterFactory, long outputLimit,
+            boolean appendOpCallbackProceedResult, byte[] searchCallbackProceedResultFalseValue,
+            byte[] searchCallbackProceedResultTrueValue) {
         super(spec, 1, 1);
         this.indexHelperFactory = indexHelperFactory;
         this.retainInput = retainInput;
@@ -82,6 +85,7 @@
         this.minFilterFieldIndexes = minFilterFieldIndexes;
         this.maxFilterFieldIndexes = maxFilterFieldIndexes;
         this.appendIndexFilter = appendIndexFilter;
+        this.nonFilterWriterFactory = nonFilterWriterFactory;
         this.outRecDescs[0] = outRecDesc;
         this.tupleFilterFactory = tupleFilterFactory;
         this.outputLimit = outputLimit;
@@ -97,8 +101,8 @@
                 recordDescProvider.getInputRecordDescriptor(getActivityId(), 0), lowKeyFields, highKeyFields,
                 lowKeyInclusive, highKeyInclusive, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
                 retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter,
-                tupleFilterFactory, outputLimit, appendOpCallbackProceedResult, searchCallbackProceedResultFalseValue,
-                searchCallbackProceedResultTrueValue);
+                nonFilterWriterFactory, tupleFilterFactory, outputLimit, appendOpCallbackProceedResult,
+                searchCallbackProceedResultFalseValue, searchCallbackProceedResultTrueValue);
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
index 77ef1dc..fb331bf 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
@@ -46,26 +46,27 @@
     public BTreeSearchOperatorNodePushable(IHyracksTaskContext ctx, int partition, RecordDescriptor inputRecDesc,
             int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
             int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, IIndexDataflowHelperFactory indexHelperFactory,
-            boolean retainInput, boolean retainMissing, IMissingWriterFactory missingWriterFactory,
-            ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter)
-            throws HyracksDataException {
+            boolean retainInput, boolean retainMissing, IMissingWriterFactory nonMatchWriterFactory,
+            ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter,
+            IMissingWriterFactory nonFilterWriterFactory) throws HyracksDataException {
         this(ctx, partition, inputRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive,
                 minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory, retainInput, retainMissing,
-                missingWriterFactory, searchCallbackFactory, appendIndexFilter, null, -1, false, null, null);
+                nonMatchWriterFactory, searchCallbackFactory, appendIndexFilter, nonFilterWriterFactory, null, -1,
+                false, null, null);
     }
 
     public BTreeSearchOperatorNodePushable(IHyracksTaskContext ctx, int partition, RecordDescriptor inputRecDesc,
             int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
             int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, IIndexDataflowHelperFactory indexHelperFactory,
-            boolean retainInput, boolean retainMissing, IMissingWriterFactory missingWriterFactory,
+            boolean retainInput, boolean retainMissing, IMissingWriterFactory nonMatchWriterFactory,
             ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter,
-            ITupleFilterFactory tupleFilterFactory, long outputLimit, boolean appendOpCallbackProceedResult,
-            byte[] searchCallbackProceedResultFalseValue, byte[] searchCallbackProceedResultTrueValue)
-            throws HyracksDataException {
+            IMissingWriterFactory nonFilterWriterFactory, ITupleFilterFactory tupleFilterFactory, long outputLimit,
+            boolean appendOpCallbackProceedResult, byte[] searchCallbackProceedResultFalseValue,
+            byte[] searchCallbackProceedResultTrueValue) throws HyracksDataException {
         super(ctx, inputRecDesc, partition, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
-                retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter,
-                tupleFilterFactory, outputLimit, appendOpCallbackProceedResult, searchCallbackProceedResultFalseValue,
-                searchCallbackProceedResultTrueValue);
+                retainInput, retainMissing, nonMatchWriterFactory, searchCallbackFactory, appendIndexFilter,
+                nonFilterWriterFactory, tupleFilterFactory, outputLimit, appendOpCallbackProceedResult,
+                searchCallbackProceedResultFalseValue, searchCallbackProceedResultTrueValue);
         this.lowKeyInclusive = lowKeyInclusive;
         this.highKeyInclusive = highKeyInclusive;
         if (lowKeyFields != null && lowKeyFields.length > 0) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorDescriptor.java
index e6637db..f8402e0 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorDescriptor.java
@@ -20,6 +20,7 @@
 package org.apache.hyracks.storage.am.btree.dataflow;
 
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
 import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -38,9 +39,10 @@
             int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
             IIndexDataflowHelperFactory dataflowHelperFactory, boolean retainInput,
             ISearchOperationCallbackFactory searchOpCallbackProvider, ITupleUpdaterFactory tupleUpdaterFactory,
-            boolean appendIndexFilter) {
+            boolean appendIndexFilter, IMissingWriterFactory nonFilterWriterFactory) {
         super(spec, outRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive, dataflowHelperFactory,
-                retainInput, false, null, searchOpCallbackProvider, null, null, appendIndexFilter);
+                retainInput, false, null, searchOpCallbackProvider, null, null, appendIndexFilter,
+                nonFilterWriterFactory);
         this.tupleUpdaterFactory = tupleUpdaterFactory;
     }
 
@@ -50,6 +52,7 @@
         return new BTreeUpdateSearchOperatorNodePushable(ctx, partition,
                 recordDescProvider.getInputRecordDescriptor(getActivityId(), 0), lowKeyFields, highKeyFields,
                 lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput, retainMissing, missingWriterFactory,
-                searchCallbackFactory, tupleUpdaterFactory.createTupleUpdater(), appendIndexFilter);
+                searchCallbackFactory, tupleUpdaterFactory.createTupleUpdater(), appendIndexFilter,
+                nonFilterWriterFactory);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java
index a296672..a102e39 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java
@@ -41,10 +41,11 @@
             int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
             IIndexDataflowHelperFactory indexHelperFactory, boolean retainInput, boolean retainMissing,
             IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
-            ITupleUpdater tupleUpdater, boolean appendIndexFilter) throws HyracksDataException {
+            ITupleUpdater tupleUpdater, boolean appendIndexFilter, IMissingWriterFactory nonFilterWriterFactory)
+            throws HyracksDataException {
         super(ctx, partition, inputRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive, null, null,
                 indexHelperFactory, retainInput, retainMissing, missingWriterFactory, searchCallbackFactory,
-                appendIndexFilter);
+                appendIndexFilter, nonFilterWriterFactory);
         this.tupleUpdater = tupleUpdater;
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/test/java/org/apache/hyracks/storage/am/btree/test/FramewriterTest.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/test/java/org/apache/hyracks/storage/am/btree/test/FramewriterTest.java
index 0600add..b42337c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/test/java/org/apache/hyracks/storage/am/btree/test/FramewriterTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/test/java/org/apache/hyracks/storage/am/btree/test/FramewriterTest.java
@@ -295,7 +295,7 @@
                             recordDescProvider.getInputRecordDescriptor(new ActivityId(new OperatorDescriptorId(0), 0),
                                     0),
                             keys, keys, lowKeyInclusive, highKeyInclusive, keys, keys, pair.getLeft(), false, false,
-                            null, pair.getRight(), false);
+                            null, pair.getRight(), false, null);
                     writers.add(writer);
                 }
             }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
index a5bc206..eab8c96 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
@@ -89,6 +89,7 @@
     protected PermutingFrameTupleReference maxFilterKey;
     protected final boolean appendIndexFilter;
     protected ArrayTupleBuilder nonFilterTupleBuild;
+    protected IMissingWriter nonFilterWriter;
     protected final ISearchOperationCallbackFactory searchCallbackFactory;
     protected boolean failed = false;
     protected IOperatorStats stats;
@@ -110,28 +111,31 @@
     // no filter and limit pushdown
     public IndexSearchOperatorNodePushable(IHyracksTaskContext ctx, RecordDescriptor inputRecDesc, int partition,
             int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, IIndexDataflowHelperFactory indexHelperFactory,
-            boolean retainInput, boolean retainMissing, IMissingWriterFactory missingWriterFactory,
-            ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter)
-            throws HyracksDataException {
+            boolean retainInput, boolean retainMissing, IMissingWriterFactory nonMatchWriterFactory,
+            ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter,
+            IMissingWriterFactory nonFilterWriterFactory) throws HyracksDataException {
         this(ctx, inputRecDesc, partition, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
-                retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter, null, -1,
-                false, null, null);
+                retainInput, retainMissing, nonMatchWriterFactory, searchCallbackFactory, appendIndexFilter,
+                nonFilterWriterFactory, null, -1, false, null, null);
     }
 
     public IndexSearchOperatorNodePushable(IHyracksTaskContext ctx, RecordDescriptor inputRecDesc, int partition,
             int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, IIndexDataflowHelperFactory indexHelperFactory,
-            boolean retainInput, boolean retainMissing, IMissingWriterFactory missingWriterFactory,
+            boolean retainInput, boolean retainMissing, IMissingWriterFactory nonMatchWriterFactory,
             ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter,
-            ITupleFilterFactory tupleFactoryFactory, long outputLimit, boolean appendSearchCallbackProceedResult,
-            byte[] searchCallbackProceedResultFalseValue, byte[] searchCallbackProceedResultTrueValue)
-            throws HyracksDataException {
+            IMissingWriterFactory nonFilterWriterFactory, ITupleFilterFactory tupleFactoryFactory, long outputLimit,
+            boolean appendSearchCallbackProceedResult, byte[] searchCallbackProceedResultFalseValue,
+            byte[] searchCallbackProceedResultTrueValue) throws HyracksDataException {
         this.ctx = ctx;
         this.indexHelper = indexHelperFactory.create(ctx.getJobletContext().getServiceContext(), partition);
         this.retainInput = retainInput;
         this.retainMissing = retainMissing;
         this.appendIndexFilter = appendIndexFilter;
-        if (this.retainMissing || this.appendIndexFilter) {
-            this.nonMatchWriter = missingWriterFactory.createMissingWriter();
+        if (this.retainMissing) {
+            this.nonMatchWriter = nonMatchWriterFactory.createMissingWriter();
+        }
+        if (this.appendIndexFilter) {
+            this.nonFilterWriter = nonFilterWriterFactory.createMissingWriter();
         }
         this.inputRecDesc = inputRecDesc;
         this.searchCallbackFactory = searchCallbackFactory;
@@ -197,7 +201,7 @@
         if (appendIndexFilter) {
             int numIndexFilterFields = index.getNumOfFilterFields();
             nonFilterTupleBuild = new ArrayTupleBuilder(numIndexFilterFields);
-            buildMissingTuple(numIndexFilterFields, nonFilterTupleBuild, nonMatchWriter);
+            buildMissingTuple(numIndexFilterFields, nonFilterTupleBuild, nonFilterWriter);
         }
 
         if (tupleFilterFactory != null) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorDescriptor.java
index 23be280..3800d17 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorDescriptor.java
@@ -41,7 +41,7 @@
             long outputLimit) {
         super(spec, outRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory,
                 retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, minFilterFieldIndexes,
-                maxFilterFieldIndexes, false, tupleFilterFactory, outputLimit, false, null, null);
+                maxFilterFieldIndexes, false, null, tupleFilterFactory, outputLimit, false, null, null);
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorNodePushable.java
index dddaab1..30813ef 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeBatchPointSearchOperatorNodePushable.java
@@ -51,7 +51,8 @@
             ITupleFilterFactory tupleFilterFactory, long outputLimit) throws HyracksDataException {
         super(ctx, partition, inputRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive,
                 minFilterKeyFields, maxFilterKeyFields, indexHelperFactory, retainInput, retainMissing,
-                missingWriterFactory, searchCallbackFactory, false, tupleFilterFactory, outputLimit, false, null, null);
+                missingWriterFactory, searchCallbackFactory, false, null, tupleFilterFactory, outputLimit, false, null,
+                null);
         this.keyFields = lowKeyFields;
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDiskComponentScanOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDiskComponentScanOperatorNodePushable.java
index 860d36e..4aa094b 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDiskComponentScanOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDiskComponentScanOperatorNodePushable.java
@@ -37,7 +37,7 @@
             RecordDescriptor inputRecDesc, IIndexDataflowHelperFactory indexHelperFactory,
             ISearchOperationCallbackFactory searchCallbackFactory) throws HyracksDataException {
         super(ctx, inputRecDesc, partition, null, null, indexHelperFactory, false, false, null, searchCallbackFactory,
-                false);
+                false, null);
     }
 
     @Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorDescriptor.java
index 2cee54d..b5b951d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorDescriptor.java
@@ -42,6 +42,7 @@
     private final int[] minFilterFieldIndexes;
     private final int[] maxFilterFieldIndexes;
     private final boolean appendIndexFilter;
+    private final IMissingWriterFactory nonFilterWriterFactory;
     private final boolean isFullTextSearchQuery;
     private final IIndexDataflowHelperFactory indexHelperFactory;
     private final IBinaryTokenizerFactory queryTokenizerFactory;
@@ -61,7 +62,7 @@
             IInvertedIndexSearchModifierFactory searchModifierFactory, boolean retainInput, boolean retainMissing,
             IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
             int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, boolean isFullTextSearchQuery, int numOfFields,
-            boolean appendIndexFilter, int frameLimit) {
+            boolean appendIndexFilter, IMissingWriterFactory nonFilterWriterFactory, int frameLimit) {
         super(spec, 1, 1);
         this.indexHelperFactory = indexHelperFactory;
         this.queryTokenizerFactory = queryTokenizerFactory;
@@ -76,6 +77,7 @@
         this.maxFilterFieldIndexes = maxFilterFieldIndexes;
         this.isFullTextSearchQuery = isFullTextSearchQuery;
         this.appendIndexFilter = appendIndexFilter;
+        this.nonFilterWriterFactory = nonFilterWriterFactory;
         this.numOfFields = numOfFields;
         this.outRecDescs[0] = outRecDesc;
         this.frameLimit = frameLimit;
@@ -89,6 +91,6 @@
                 recordDescProvider.getInputRecordDescriptor(getActivityId(), 0), partition, minFilterFieldIndexes,
                 maxFilterFieldIndexes, indexHelperFactory, retainInput, retainMissing, missingWriterFactory,
                 searchCallbackFactory, searchModifier, queryTokenizerFactory, fullTextConfigEvaluatorFactory,
-                queryField, isFullTextSearchQuery, numOfFields, appendIndexFilter, frameLimit);
+                queryField, isFullTextSearchQuery, numOfFields, appendIndexFilter, nonFilterWriterFactory, frameLimit);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorNodePushable.java
index e4dbb43..571ae5c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorNodePushable.java
@@ -62,10 +62,11 @@
             IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
             IInvertedIndexSearchModifier searchModifier, IBinaryTokenizerFactory binaryTokenizerFactory,
             IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory, int queryFieldIndex,
-            boolean isFullTextSearchQuery, int numOfFields, boolean appendIndexFilter, int frameLimit)
-            throws HyracksDataException {
+            boolean isFullTextSearchQuery, int numOfFields, boolean appendIndexFilter,
+            IMissingWriterFactory nonFilterWriterFactory, int frameLimit) throws HyracksDataException {
         super(ctx, inputRecDesc, partition, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
-                retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter);
+                retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter,
+                nonFilterWriterFactory);
         this.searchModifier = searchModifier;
         this.binaryTokenizerFactory = binaryTokenizerFactory;
         this.fullTextConfigEvaluatorFactory = fullTextConfigEvaluatorFactory;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
index 46727cd..cc6d76d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
@@ -39,6 +39,7 @@
     protected final int[] minFilterFieldIndexes;
     protected final int[] maxFilterFieldIndexes;
     protected final boolean appendIndexFilter;
+    protected final IMissingWriterFactory nonFilterWriterFactory;
     protected final IIndexDataflowHelperFactory indexHelperFactory;
     protected final boolean retainInput;
     protected final boolean retainMissing;
@@ -52,18 +53,19 @@
             boolean lowKeyInclusive, boolean highKeyInclusive, IIndexDataflowHelperFactory indexHelperFactory,
             boolean retainInput, boolean retainMissing, IMissingWriterFactory missingWriterFactory,
             ISearchOperationCallbackFactory searchCallbackFactory, int[] minFilterFieldIndexes,
-            int[] maxFilterFieldIndexes, boolean appendIndexFilter) {
+            int[] maxFilterFieldIndexes, boolean appendIndexFilter, IMissingWriterFactory nonFilterWriterFactory) {
         this(spec, outRecDesc, keyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput,
                 retainMissing, missingWriterFactory, searchCallbackFactory, minFilterFieldIndexes,
-                maxFilterFieldIndexes, appendIndexFilter, false, null, null);
+                maxFilterFieldIndexes, appendIndexFilter, nonFilterWriterFactory, false, null, null);
     }
 
     public RTreeSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor outRecDesc, int[] keyFields,
             boolean lowKeyInclusive, boolean highKeyInclusive, IIndexDataflowHelperFactory indexHelperFactory,
             boolean retainInput, boolean retainMissing, IMissingWriterFactory missingWriterFactory,
             ISearchOperationCallbackFactory searchCallbackFactory, int[] minFilterFieldIndexes,
-            int[] maxFilterFieldIndexes, boolean appendIndexFilter, boolean appendOpCallbackProceedResult,
-            byte[] searchCallbackProceedResultFalseValue, byte[] searchCallbackProceedResultTrueValue) {
+            int[] maxFilterFieldIndexes, boolean appendIndexFilter, IMissingWriterFactory nonFilterWriterFactory,
+            boolean appendOpCallbackProceedResult, byte[] searchCallbackProceedResultFalseValue,
+            byte[] searchCallbackProceedResultTrueValue) {
         super(spec, 1, 1);
         this.indexHelperFactory = indexHelperFactory;
         this.retainInput = retainInput;
@@ -76,6 +78,7 @@
         this.minFilterFieldIndexes = minFilterFieldIndexes;
         this.maxFilterFieldIndexes = maxFilterFieldIndexes;
         this.appendIndexFilter = appendIndexFilter;
+        this.nonFilterWriterFactory = nonFilterWriterFactory;
         this.outRecDescs[0] = outRecDesc;
         this.appendOpCallbackProceedResult = appendOpCallbackProceedResult;
         this.searchCallbackProceedResultFalseValue = searchCallbackProceedResultFalseValue;
@@ -88,7 +91,7 @@
         return new RTreeSearchOperatorNodePushable(ctx, partition,
                 recordDescProvider.getInputRecordDescriptor(getActivityId(), 0), keyFields, minFilterFieldIndexes,
                 maxFilterFieldIndexes, indexHelperFactory, retainInput, retainMissing, missingWriterFactory,
-                searchCallbackFactory, appendIndexFilter, appendOpCallbackProceedResult,
+                searchCallbackFactory, appendIndexFilter, nonFilterWriterFactory, appendOpCallbackProceedResult,
                 searchCallbackProceedResultFalseValue, searchCallbackProceedResultTrueValue);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
index 4d460b1..a8a4252 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
@@ -42,23 +42,23 @@
             int[] keyFields, int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes,
             IIndexDataflowHelperFactory indexHelperFactory, boolean retainInput, boolean retainMissing,
             IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
-            boolean appendIndexFilter) throws HyracksDataException {
+            boolean appendIndexFilter, IMissingWriterFactory nonFilterWriterFactory) throws HyracksDataException {
         this(ctx, partition, inputRecDesc, keyFields, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
-                retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter, false, null,
-                null);
+                retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter,
+                nonFilterWriterFactory, false, null, null);
     }
 
     public RTreeSearchOperatorNodePushable(IHyracksTaskContext ctx, int partition, RecordDescriptor inputRecDesc,
             int[] keyFields, int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes,
             IIndexDataflowHelperFactory indexHelperFactory, boolean retainInput, boolean retainMissing,
             IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
-            boolean appendIndexFilter, boolean appendOpCallbackProceedResult,
-            byte[] searchCallbackProceedResultFalseValue, byte[] searchCallbackProceedResultTrueValue)
-            throws HyracksDataException {
+            boolean appendIndexFilter, IMissingWriterFactory nonFilterWriterFactory,
+            boolean appendOpCallbackProceedResult, byte[] searchCallbackProceedResultFalseValue,
+            byte[] searchCallbackProceedResultTrueValue) throws HyracksDataException {
         // TODO: predicate & limit pushdown not enabled for RTree yet
         super(ctx, inputRecDesc, partition, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
-                retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter, null, -1,
-                appendOpCallbackProceedResult, searchCallbackProceedResultFalseValue,
+                retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter,
+                nonFilterWriterFactory, null, -1, appendOpCallbackProceedResult, searchCallbackProceedResultFalseValue,
                 searchCallbackProceedResultTrueValue);
         if (keyFields != null && keyFields.length > 0) {
             searchKey = new PermutingFrameTupleReference();