PHOENIX-4287 Addendum to correctly set useStatsForParallelization property
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
index 25d4194..b5e4588 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
@@ -17,7 +17,6 @@
  */
 package org.apache.phoenix.end2end;
 
-import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_USE_STATS_FOR_PARALLELIZATION;
 import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -34,6 +33,7 @@
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixResultSet;
 import org.apache.phoenix.query.BaseTest;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.TableNotFoundException;
@@ -352,7 +352,7 @@
     }
 
     @Test
-    public void testSettingUseStatsForQueryPlanProperty() throws Exception {
+    public void testSettingUseStatsForParallelizationProperty() throws Exception {
         try (Connection conn = DriverManager.getConnection(getUrl())) {
             String table = generateUniqueName();
             String ddl =
@@ -360,20 +360,31 @@
                             + " (PK1 INTEGER NOT NULL PRIMARY KEY, KV1 VARCHAR) USE_STATS_FOR_PARALLELIZATION = false";
             conn.createStatement().execute(ddl);
             assertUseStatsForQueryFlag(table, conn.unwrap(PhoenixConnection.class), false);
+
             ddl = "ALTER TABLE " + table + " SET USE_STATS_FOR_PARALLELIZATION = true";
             conn.createStatement().execute(ddl);
             assertUseStatsForQueryFlag(table, conn.unwrap(PhoenixConnection.class), true);
+
+            table = generateUniqueName();
+            ddl =
+                    "CREATE TABLE " + table
+                            + " (PK1 INTEGER NOT NULL PRIMARY KEY, KV1 VARCHAR) USE_STATS_FOR_PARALLELIZATION = false";
+            conn.createStatement().execute(ddl);
+            assertUseStatsForQueryFlag(table, conn.unwrap(PhoenixConnection.class), false);
+
             table = generateUniqueName();
             ddl = "CREATE TABLE " + table + " (PK1 INTEGER NOT NULL PRIMARY KEY, KV1 VARCHAR)";
             conn.createStatement().execute(ddl);
-            assertUseStatsForQueryFlag(table, conn.unwrap(PhoenixConnection.class),
-                DEFAULT_USE_STATS_FOR_PARALLELIZATION);
+
+            // because we didn't set the property, PTable.useStatsForParallelization() should return
+            // null
+            assertUseStatsForQueryFlag(table, conn.unwrap(PhoenixConnection.class), null);
         }
     }
 
     private static void assertUseStatsForQueryFlag(String tableName, PhoenixConnection conn,
-            boolean flag) throws TableNotFoundException, SQLException {
-        assertEquals(flag,
+            Boolean expected) throws TableNotFoundException, SQLException {
+        assertEquals(expected,
             conn.unwrap(PhoenixConnection.class).getMetaDataCache()
                     .getTableRef(new PTableKey(null, tableName)).getTable()
                     .useStatsForParallelization());
@@ -383,7 +394,12 @@
         stmt.setString(1, tableName);
         ResultSet rs = stmt.executeQuery();
         rs.next();
-        assertEquals(flag, rs.getBoolean(1));
+        boolean b = rs.getBoolean(1);
+        if (expected == null) {
+            assertTrue(rs.wasNull());
+        } else {
+            assertEquals(expected, b);
+        }
     }
 
     @Test
@@ -510,8 +526,7 @@
             int guidePostWidth = 20;
             String ddl =
                     "CREATE TABLE " + tableName + " (k INTEGER PRIMARY KEY, a bigint, b bigint)"
-                            + " GUIDE_POSTS_WIDTH=" + guidePostWidth
-                            + ", USE_STATS_FOR_PARALLELIZATION=false";
+                            + " GUIDE_POSTS_WIDTH=" + guidePostWidth;
             byte[][] splits =
                     new byte[][] { Bytes.toBytes(102), Bytes.toBytes(105), Bytes.toBytes(108) };
             BaseTest.createTestTable(getUrl(), ddl, splits, null);
@@ -531,18 +546,70 @@
         List<Object> binds = Lists.newArrayList();
         try (Connection conn = DriverManager.getConnection(getUrl())) {
             String sql = "SELECT COUNT(*) " + " FROM " + tableName;
+            // We don't have the use stats for parallelization property
+            // set on the table. In this case, we end up defaulting to the
+            // value set in config which is true.
             ResultSet rs = conn.createStatement().executeQuery(sql);
+            // stats are being used for parallelization. So number of scans is higher.
+            assertEquals(14, rs.unwrap(PhoenixResultSet.class).getStatement().getQueryPlan()
+                    .getScans().get(0).size());
             assertTrue(rs.next());
             assertEquals(10, rs.getInt(1));
             Estimate info = getByteRowEstimates(conn, sql, binds);
             assertEquals((Long) 10l, info.getEstimatedRows());
             assertTrue(info.getEstimateInfoTs() > 0);
+            
+            // Now, let's disable USE_STATS_FOR_PARALLELIZATION on the table
+            conn.createStatement().execute("ALTER TABLE " + tableName + " SET USE_STATS_FOR_PARALLELIZATION = " + false);
+            rs = conn.createStatement().executeQuery(sql);
+            // stats are not being used for parallelization. So number of scans is lower.
+            assertEquals(4, rs.unwrap(PhoenixResultSet.class).getStatement().getQueryPlan()
+                    .getScans().get(0).size());
+            assertTrue(rs.next());
+            assertEquals(10, rs.getInt(1));
+            info = getByteRowEstimates(conn, sql, binds);
+            assertEquals((Long) 10l, info.getEstimatedRows());
+            assertTrue(info.getEstimateInfoTs() > 0);
+            
+            // assert that the aggregate query on view also works correctly
+            String viewName = "V_" + generateUniqueName();
+            conn.createStatement()
+                    .execute("CREATE VIEW " + viewName + " AS SELECT * FROM " + tableName + " USE_STATS_FOR_PARALLELIZATION = false");
+            sql = "SELECT COUNT(*) FROM " + viewName;
+            rs = conn.createStatement().executeQuery(sql);
+            // stats are not being used for parallelization. So number of scans is lower.
+            assertEquals(4, rs.unwrap(PhoenixResultSet.class).getStatement().getQueryPlan()
+                    .getScans().get(0).size());
+            assertTrue(rs.next());
+            assertEquals(10, rs.getInt(1));
+            info = getByteRowEstimates(conn, sql, binds);
+            assertEquals((Long) 10l, info.getEstimatedRows());
+            assertTrue(info.getEstimateInfoTs() > 0);
 
             // Now let's make sure that when using stats for parallelization, our estimates
-            // and query results stay the same
+            // and query results stay the same for view and base table
             conn.createStatement().execute(
                 "ALTER TABLE " + tableName + " SET USE_STATS_FOR_PARALLELIZATION=true");
+            sql = "SELECT COUNT(*) FROM " + tableName;
+            // query the table
             rs = conn.createStatement().executeQuery(sql);
+            // stats are being used for parallelization. So number of scans is higher.
+            assertEquals(14, rs.unwrap(PhoenixResultSet.class).getStatement().getQueryPlan()
+                .getScans().get(0).size());
+            assertTrue(rs.next());
+            assertEquals(10, rs.getInt(1));
+            info = getByteRowEstimates(conn, sql, binds);
+            assertEquals((Long) 10l, info.getEstimatedRows());
+            assertTrue(info.getEstimateInfoTs() > 0);
+
+            conn.createStatement().execute(
+                "ALTER TABLE " + viewName + " SET USE_STATS_FOR_PARALLELIZATION=true");
+            sql = "SELECT COUNT(*) FROM " + viewName;
+            // query the view
+            rs = conn.createStatement().executeQuery(sql);
+            // stats are not being used for parallelization. So number of scans is higher.
+            assertEquals(14, rs.unwrap(PhoenixResultSet.class).getStatement().getQueryPlan()
+                .getScans().get(0).size());
             assertTrue(rs.next());
             assertEquals(10, rs.getInt(1));
             info = getByteRowEstimates(conn, sql, binds);
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
index afcc056..d05ab79 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
@@ -960,7 +960,7 @@
                 .fromSerializedValue((byte)PTinyint.INSTANCE.toObject(encodingSchemeKv.getValueArray(),
                     encodingSchemeKv.getValueOffset(), encodingSchemeKv.getValueLength()));
         Cell useStatsForParallelizationKv = tableKeyValues[USE_STATS_FOR_PARALLELIZATION_INDEX];
-        boolean useStatsForParallelization = useStatsForParallelizationKv == null ? true : Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(useStatsForParallelizationKv.getValueArray(), useStatsForParallelizationKv.getValueOffset(), useStatsForParallelizationKv.getValueLength()));
+        Boolean useStatsForParallelization = useStatsForParallelizationKv == null ? null : Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(useStatsForParallelizationKv.getValueArray(), useStatsForParallelizationKv.getValueOffset(), useStatsForParallelizationKv.getValueLength()));
         
         List<PColumn> columns = Lists.newArrayListWithExpectedSize(columnCount);
         List<PTable> indexes = Lists.newArrayList();
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
index e9deec3..b4c9698 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
@@ -35,7 +35,6 @@
 import java.io.DataInput;
 import java.io.DataInputStream;
 import java.io.EOFException;
-import java.io.IOException;
 import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.BitSet;
@@ -152,7 +151,7 @@
     private Long estimateInfoTimestamp;
     private boolean hasGuidePosts;
     private Scan scan;
-    private boolean useStatsForParallelization;
+    private final boolean useStatsForParallelization;
     protected Map<ImmutableBytesPtr,ServerCache> caches;
     
     static final Function<HRegionLocation, KeyRange> TO_KEY_RANGE = new Function<HRegionLocation, KeyRange>() {
@@ -492,7 +491,11 @@
         scanId = new UUID(ThreadLocalRandom.current().nextLong(), ThreadLocalRandom.current().nextLong()).toString();
         
         initializeScan(plan, perScanLimit, offset, scan);
-        this.useStatsForParallelization = table.useStatsForParallelization();
+        this.useStatsForParallelization =
+                table.useStatsForParallelization() == null
+                        ? context.getConnection().getQueryServices().getConfiguration().getBoolean(
+                            USE_STATS_FOR_PARALLELIZATION, DEFAULT_USE_STATS_FOR_PARALLELIZATION)
+                        : table.useStatsForParallelization();
         this.scans = getParallelScans();
         List<KeyRange> splitRanges = Lists.newArrayListWithExpectedSize(scans.size() * ESTIMATED_GUIDEPOSTS_PER_REGION);
         for (List<Scan> scanList : scans) {
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateTable.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateTable.java
index 3da27a7..8f15c5e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateTable.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateTable.java
@@ -312,7 +312,7 @@
     }
 
     @Override
-    public boolean useStatsForParallelization() {
+    public Boolean useStatsForParallelization() {
         return delegate.useStatsForParallelization();
     }
 }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
index 701633b..7ce2167 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
@@ -2002,15 +2002,8 @@
                 }
             }
 
-            boolean useStatsForParallelization =
-                    connection.getQueryServices().getProps().getBoolean(
-                        QueryServices.USE_STATS_FOR_PARALLELIZATION,
-                        QueryServicesOptions.DEFAULT_USE_STATS_FOR_PARALLELIZATION);
             Boolean useStatsForParallelizationProp =
                     (Boolean) TableProperty.USE_STATS_FOR_PARALLELIZATION.getValue(tableProps);
-            if (useStatsForParallelizationProp != null) {
-                useStatsForParallelization = useStatsForParallelizationProp;
-            }
 
             boolean sharedTable = statement.getTableType() == PTableType.VIEW || allocateIndexId;
             if (transactional) {
@@ -2606,7 +2599,11 @@
             }
             tableUpsert.setByte(26, immutableStorageScheme.getSerializedMetadataValue());
             tableUpsert.setByte(27, encodingScheme.getSerializedMetadataValue());
-            tableUpsert.setBoolean(28, useStatsForParallelization);
+            if (useStatsForParallelizationProp == null) {
+                tableUpsert.setNull(28, Types.BOOLEAN);
+            } else {
+                tableUpsert.setBoolean(28, useStatsForParallelizationProp);
+            }
             tableUpsert.execute();
 
             if (asyncCreatedDate != null) {
@@ -2711,7 +2708,7 @@
                         PTable.INITIAL_SEQ_NUM, pkName == null ? null : PNameFactory.newName(pkName), saltBucketNum, columns.values(),
                         parent == null ? null : parent.getSchemaName(), parent == null ? null : parent.getTableName(), Collections.<PTable>emptyList(), isImmutableRows,
                         physicalNames, defaultFamilyName == null ? null : PNameFactory.newName(defaultFamilyName), viewStatement, Boolean.TRUE.equals(disableWAL), multiTenant, storeNulls, viewType,
-                        result.getViewIndexId(), indexType, rowKeyOrderOptimizable, transactional, updateCacheFrequency, 0L, isNamespaceMapped, autoPartitionSeq, isAppendOnlySchema, immutableStorageScheme, encodingScheme, cqCounterToBe, useStatsForParallelization);
+                        result.getViewIndexId(), indexType, rowKeyOrderOptimizable, transactional, updateCacheFrequency, 0L, isNamespaceMapped, autoPartitionSeq, isAppendOnlySchema, immutableStorageScheme, encodingScheme, cqCounterToBe, useStatsForParallelizationProp);
                 result = new MetaDataMutationResult(code, result.getMutationTime(), table, true);
                 addTableToCache(result);
                 return table;
@@ -3304,11 +3301,12 @@
                     }
                 }
                 Boolean useStatsForParallelization = null;
-                if (useStatsForParallelizationProp != null) {
-                    if (useStatsForParallelizationProp.booleanValue() != table.useStatsForParallelization()) {
-                        useStatsForParallelization = useStatsForParallelizationProp;
-                        changingPhoenixTableProperty = true;
-                    }
+                if (useStatsForParallelizationProp != null
+                        && (table.useStatsForParallelization() == null
+                                || (useStatsForParallelizationProp.booleanValue() != table
+                                        .useStatsForParallelization()))) {
+                    useStatsForParallelization = useStatsForParallelizationProp;
+                    changingPhoenixTableProperty = true;
                 }
                 Boolean isTransactional = null;
                 if (isTransactionalProp != null) {
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
index d59e785..ec931b7 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
@@ -705,7 +705,7 @@
     ImmutableStorageScheme getImmutableStorageScheme();
     QualifierEncodingScheme getEncodingScheme();
     EncodedCQCounter getEncodedCQCounter();
-    boolean useStatsForParallelization();
+    Boolean useStatsForParallelization();
     
     /**
      * Class to help track encoded column qualifier counters per column family.
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
index ae91d17..430b163 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
@@ -151,7 +151,7 @@
     private ImmutableStorageScheme immutableStorageScheme;
     private QualifierEncodingScheme qualifierEncodingScheme;
     private EncodedCQCounter encodedCQCounter;
-    private boolean useStatsForParallelization;
+    private Boolean useStatsForParallelization;
 
     public PTableImpl() {
         this.indexes = Collections.emptyList();
@@ -184,7 +184,7 @@
         this.isNamespaceMapped = isNamespaceMapped;
     }
     
-    public PTableImpl(PName tenantId, String schemaName, String tableName, long timestamp, List<PColumnFamily> families, boolean isNamespaceMapped, ImmutableStorageScheme storageScheme, QualifierEncodingScheme encodingScheme, boolean useStatsForParallelization) { // For base table of mapped VIEW
+    public PTableImpl(PName tenantId, String schemaName, String tableName, long timestamp, List<PColumnFamily> families, boolean isNamespaceMapped, ImmutableStorageScheme storageScheme, QualifierEncodingScheme encodingScheme, Boolean useStatsForParallelization) { // For base table of mapped VIEW
         Preconditions.checkArgument(tenantId==null || tenantId.getBytes().length > 0); // tenantId should be null or not empty
         this.tenantId = tenantId;
         this.name = PNameFactory.newName(SchemaUtil.getTableName(schemaName, tableName));
@@ -214,7 +214,7 @@
     // For indexes stored in shared physical tables
     public PTableImpl(PName tenantId, PName schemaName, PName tableName, long timestamp, List<PColumnFamily> families, 
             List<PColumn> columns, List<PName> physicalNames, Short viewIndexId, boolean multiTenant, boolean isNamespaceMpped, ImmutableStorageScheme storageScheme, QualifierEncodingScheme qualifierEncodingScheme, 
-            EncodedCQCounter encodedCQCounter, boolean useStatsForParallelization) throws SQLException {
+            EncodedCQCounter encodedCQCounter, Boolean useStatsForParallelization) throws SQLException {
         this.pkColumns = this.allColumns = Collections.emptyList();
         this.rowKeySchema = RowKeySchema.EMPTY_SCHEMA;
         this.indexes = Collections.emptyList();
@@ -375,7 +375,7 @@
             boolean isImmutableRows, List<PName> physicalNames, PName defaultFamilyName, String viewExpression,
             boolean disableWAL, boolean multiTenant, boolean storeNulls, ViewType viewType, Short viewIndexId,
             IndexType indexType, boolean rowKeyOrderOptimizable, boolean isTransactional, long updateCacheFrequency,
-            long indexDisableTimestamp, boolean isNamespaceMapped, String autoPartitionSeqName, boolean isAppendOnlySchema, ImmutableStorageScheme storageScheme, QualifierEncodingScheme qualifierEncodingScheme, EncodedCQCounter encodedCQCounter, boolean useStatsForParallelization) throws SQLException {
+            long indexDisableTimestamp, boolean isNamespaceMapped, String autoPartitionSeqName, boolean isAppendOnlySchema, ImmutableStorageScheme storageScheme, QualifierEncodingScheme qualifierEncodingScheme, EncodedCQCounter encodedCQCounter, Boolean useStatsForParallelization) throws SQLException {
         return new PTableImpl(tenantId, schemaName, tableName, type, state, timeStamp, sequenceNumber, pkName, bucketNum, columns, dataSchemaName,
                 dataTableName, indexes, isImmutableRows, physicalNames, defaultFamilyName,
                 viewExpression, disableWAL, multiTenant, storeNulls, viewType, viewIndexId,
@@ -391,7 +391,7 @@
             IndexType indexType, boolean rowKeyOrderOptimizable, boolean isTransactional, long updateCacheFrequency,
             int baseColumnCount, long indexDisableTimestamp, boolean isNamespaceMapped,
             String autoPartitionSeqName, boolean isAppendOnlySchema, ImmutableStorageScheme storageScheme,
-            QualifierEncodingScheme qualifierEncodingScheme, EncodedCQCounter encodedCQCounter, boolean useStatsForParallelization)
+            QualifierEncodingScheme qualifierEncodingScheme, EncodedCQCounter encodedCQCounter, Boolean useStatsForParallelization)
             throws SQLException {
         return new PTableImpl(tenantId, schemaName, tableName, type, state, timeStamp, sequenceNumber, pkName,
                 bucketNum, columns, dataSchemaName, dataTableName, indexes, isImmutableRows, physicalNames,
@@ -407,7 +407,7 @@
             boolean storeNulls, ViewType viewType, Short viewIndexId, IndexType indexType,
             int baseColumnCount, boolean rowKeyOrderOptimizable, boolean isTransactional, long updateCacheFrequency,
             long indexDisableTimestamp, boolean isNamespaceMapped, String autoPartitionSeqName, boolean isAppendOnlySchema, ImmutableStorageScheme storageScheme, 
-            QualifierEncodingScheme qualifierEncodingScheme, EncodedCQCounter encodedCQCounter, boolean useStatsForParallelization) throws SQLException {
+            QualifierEncodingScheme qualifierEncodingScheme, EncodedCQCounter encodedCQCounter, Boolean useStatsForParallelization) throws SQLException {
         init(tenantId, schemaName, tableName, type, state, timeStamp, sequenceNumber, pkName, bucketNum, columns,
                 parentSchemaName, parentTableName, indexes, isImmutableRows, physicalNames, defaultFamilyName,
                 viewExpression, disableWAL, multiTenant, storeNulls, viewType, viewIndexId, indexType, baseColumnCount, rowKeyOrderOptimizable,
@@ -447,7 +447,7 @@
             boolean multiTenant, boolean storeNulls, ViewType viewType, Short viewIndexId,
             IndexType indexType , int baseColumnCount, boolean rowKeyOrderOptimizable, boolean isTransactional, long updateCacheFrequency, long indexDisableTimestamp, 
             boolean isNamespaceMapped, String autoPartitionSeqName, boolean isAppendOnlySchema, ImmutableStorageScheme storageScheme, QualifierEncodingScheme qualifierEncodingScheme, 
-            EncodedCQCounter encodedCQCounter, boolean useStatsForParallelization) throws SQLException {
+            EncodedCQCounter encodedCQCounter, Boolean useStatsForParallelization) throws SQLException {
         Preconditions.checkNotNull(schemaName);
         Preconditions.checkArgument(tenantId==null || tenantId.getBytes().length > 0); // tenantId should be null or not empty
         int estimatedSize = SizedUtil.OBJECT_SIZE * 2 + 23 * SizedUtil.POINTER_SIZE + 4 * SizedUtil.INT_SIZE + 2 * SizedUtil.LONG_SIZE + 2 * SizedUtil.INT_OBJECT_SIZE +
@@ -1344,7 +1344,7 @@
         		}
         	}
         }
-        boolean useStatsForParallelization = true;
+        Boolean useStatsForParallelization = null;
         if (table.hasUseStatsForParallelization()) {
             useStatsForParallelization = table.getUseStatsForParallelization();
         }
@@ -1456,7 +1456,9 @@
       if (table.getEncodingScheme() != null) {
           builder.setEncodingScheme(ByteStringer.wrap(new byte[]{table.getEncodingScheme().getSerializedMetadataValue()}));
       }
-      builder.setUseStatsForParallelization(table.useStatsForParallelization());
+      if (table.useStatsForParallelization() != null) {
+          builder.setUseStatsForParallelization(table.useStatsForParallelization());
+      }
       return builder.build();
     }
 
@@ -1543,7 +1545,7 @@
     }
     
     @Override
-    public boolean useStatsForParallelization() {
+    public Boolean useStatsForParallelization() {
         return useStatsForParallelization;
     }