Repository: phoenix
Updated Branches:
  refs/heads/master 78656a53e -> c5edd3a71


PHOENIX-1578 Add DEFAULT_STORE_NULLS_ATTRIB

Add DEFAULT_STORE_NULLS_ATTRIB configuration key to set the
default STORE_NULLS behavior for new table creation.


Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo
Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/c5edd3a7
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/c5edd3a7
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/c5edd3a7

Branch: refs/heads/master
Commit: c5edd3a71de6e817f367a34afe57d212bd06ec90
Parents: 4564401
Author: Gabriel Reid <gabri...@ngdata.com>
Authored: Tue Jan 13 16:56:10 2015 +0100
Committer: Gabriel Reid <gabri...@ngdata.com>
Committed: Wed Jan 14 08:26:33 2015 +0100

----------------------------------------------------------------------
 .../apache/phoenix/end2end/StoreNullsIT.java    |  21 ++-
 .../query/ConnectionQueryServicesImpl.java      | 155 +++++++++----------
 .../org/apache/phoenix/query/QueryServices.java |  37 ++---
 .../phoenix/query/QueryServicesOptions.java     | 121 ++++++++-------
 .../apache/phoenix/schema/MetaDataClient.java   |   9 +-
 5 files changed, 183 insertions(+), 160 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/c5edd3a7/phoenix-core/src/it/java/org/apache/phoenix/end2end/StoreNullsIT.java
----------------------------------------------------------------------
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StoreNullsIT.java 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StoreNullsIT.java
index c834ade..8efeefc 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StoreNullsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StoreNullsIT.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.SchemaUtil;
 import org.junit.After;
@@ -156,6 +157,23 @@ public class StoreNullsIT extends BaseHBaseManagedTimeIT {
         rs.close();
     }
 
+    @Test
+    public void testSetStoreNullsDefaultViaConfig() throws SQLException {
+        Properties props = new Properties();
+        props.setProperty(QueryServices.DEFAULT_STORE_NULLS_ATTRIB, "true");
+        Connection storeNullsConn = DriverManager.getConnection(getUrl(), 
props);
+
+        Statement stmt = storeNullsConn.createStatement();
+        stmt.execute("CREATE TABLE with_nulls_default (" +
+                "id smallint primary key," +
+                "name varchar)");
+
+        ResultSet rs = stmt.executeQuery("SELECT store_nulls FROM 
SYSTEM.CATALOG " +
+                "WHERE table_name = 'WITH_NULLS_DEFAULT' AND store_nulls is 
not null");
+        assertTrue(rs.next());
+        assertTrue(rs.getBoolean(1));
+    }
+
     /**
      * Runs a major compaction, and then waits until the compaction is 
complete before returning.
      *
@@ -202,6 +220,7 @@ public class StoreNullsIT extends BaseHBaseManagedTimeIT {
         }
 
         htable.close();
-
     }
+
+
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c5edd3a7/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
----------------------------------------------------------------------
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index 8050b4a..d9490ce 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -168,27 +168,27 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
     private final String userName;
     private final 
ConcurrentHashMap<ImmutableBytesWritable,ConnectionQueryServices> childServices;
     private final Cache<ImmutableBytesPtr, PTableStats> tableStatsCache;
-    
+
     // Cache the latest meta data here for future connections
     // writes guarded by "latestMetaDataLock"
     private volatile PMetaData latestMetaData;
     private final Object latestMetaDataLock = new Object();
-    
+
     // Lowest HBase version on the cluster.
     private int lowestClusterHBaseVersion = Integer.MAX_VALUE;
     private boolean hasInvalidIndexConfiguration = false;
-    
+
     @GuardedBy("connectionCountLock")
     private int connectionCount = 0;
     private final Object connectionCountLock = new Object();
-    
+
     private HConnection connection;
     private volatile boolean initialized;
     private volatile int nSequenceSaltBuckets;
-    
+
     // writes guarded by "this"
     private volatile boolean closed;
-    
+
     private volatile SQLException initializationException;
     // setting this member variable guarded by "connectionCountLock"
     private volatile ConcurrentMap<SequenceKey,Sequence> sequenceMap = 
Maps.newConcurrentMap();
@@ -244,7 +244,7 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
                 .expireAfterWrite(halfStatsUpdateFreq, TimeUnit.MILLISECONDS)
                 .build();
     }
-    
+
     private void openConnection() throws SQLException {
         try {
             // check if we need to authenticate with kerberos
@@ -276,21 +276,21 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
             throw new 
TableNotFoundException(Bytes.toString(schemaAndTableName[0]), 
Bytes.toString(schemaAndTableName[1]));
         } catch (IOException e) {
                throw new SQLException(e);
-        } 
+        }
     }
-    
+
     @Override
     public HTableDescriptor getTableDescriptor(byte[] tableName) throws 
SQLException {
         HTableInterface htable = getTable(tableName);
         try {
             return htable.getTableDescriptor();
         } catch (IOException e) {
-            if(e instanceof org.apache.hadoop.hbase.TableNotFoundException || 
+            if(e instanceof org.apache.hadoop.hbase.TableNotFoundException ||
                 e.getCause() instanceof 
org.apache.hadoop.hbase.TableNotFoundException) {
               byte[][] schemaAndTableName = new byte[2][];
               SchemaUtil.getVarChars(tableName, schemaAndTableName);
               throw new 
TableNotFoundException(Bytes.toString(schemaAndTableName[0]), 
Bytes.toString(schemaAndTableName[1]));
-            } 
+            }
             throw new RuntimeException(e);
         } finally {
             Closeables.closeQuietly(htable);
@@ -355,7 +355,7 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
                 }
             }
         }
-    }    
+    }
 
     protected ConnectionQueryServices newChildQueryService() {
         return new ChildQueryServices(this);
@@ -382,12 +382,12 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
     public void clearTableRegionCache(byte[] tableName) throws SQLException {
         connection.clearRegionCache(TableName.valueOf(tableName));
     }
-    
+
     @Override
     public List<HRegionLocation> getAllTableRegions(byte[] tableName) throws 
SQLException {
         /*
          * Use HConnection.getRegionLocation as it uses the cache in 
HConnection, while getting
-         * all region locations from the HTable doesn't. 
+         * all region locations from the HTable doesn't.
          */
         int retryCount = 0, maxRetryCount = 1;
         boolean reload =false;
@@ -427,8 +427,8 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
                 // If existing table isn't older than new table, don't replace
                 // If a client opens a connection at an earlier timestamp, 
this can happen
                 PTable existingTable = latestMetaData.getTable(new 
PTableKey(table.getTenantId(), table.getName().getString()));
-                if (existingTable.getTimeStamp() >= table.getTimeStamp()) { 
-                    return latestMetaData; 
+                if (existingTable.getTimeStamp() >= table.getTimeStamp()) {
+                    return latestMetaData;
                 }
             } catch (TableNotFoundException e) {}
             latestMetaData = latestMetaData.addTable(table);
@@ -569,11 +569,10 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
         hcd.setValue(key, value == null ? null : value.toString());
       }
     }
-    
-    
+
     private HTableDescriptor generateTableDescriptor(byte[] tableName, 
HTableDescriptor existingDesc, PTableType tableType, Map<String,Object> 
tableProps, List<Pair<byte[],Map<String,Object>>> families, byte[][] splits) 
throws SQLException {
-        String defaultFamilyName = 
(String)tableProps.remove(PhoenixDatabaseMetaData.DEFAULT_COLUMN_FAMILY_NAME);  
              
-        HTableDescriptor tableDescriptor = (existingDesc != null) ? new 
HTableDescriptor(existingDesc) : 
+        String defaultFamilyName = 
(String)tableProps.remove(PhoenixDatabaseMetaData.DEFAULT_COLUMN_FAMILY_NAME);
+        HTableDescriptor tableDescriptor = (existingDesc != null) ? new 
HTableDescriptor(existingDesc) :
           new HTableDescriptor(TableName.valueOf(tableName));
         for (Entry<String,Object> entry : tableProps.entrySet()) {
             String key = entry.getKey();
@@ -583,7 +582,7 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
         if (families.isEmpty()) {
             if (tableType != PTableType.VIEW) {
                 byte[] defaultFamilyByes = defaultFamilyName == null ? 
QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES : Bytes.toBytes(defaultFamilyName);
-                // Add dummy column family so we have key values for tables 
that 
+                // Add dummy column family so we have key values for tables 
that
                 HColumnDescriptor columnDescriptor = 
generateColumnFamilyDescriptor(new 
Pair<byte[],Map<String,Object>>(defaultFamilyByes,Collections.<String,Object>emptyMap()),
 tableType);
                 tableDescriptor.addFamily(columnDescriptor);
             }
@@ -633,9 +632,9 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
             // Since indexes can't have indexes, don't install our indexing 
coprocessor for indexes.
             // Also don't install on the SYSTEM.CATALOG and SYSTEM.STATS table 
because we use
             // all-or-none mutate class which break when this coprocessor is 
installed (PHOENIX-1318).
-            if ((tableType != PTableType.INDEX && tableType != 
PTableType.VIEW) 
+            if ((tableType != PTableType.INDEX && tableType != PTableType.VIEW)
                     && !SchemaUtil.isMetaTable(tableName)
-                    && !SchemaUtil.isStatsTable(tableName) 
+                    && !SchemaUtil.isStatsTable(tableName)
                     && !descriptor.hasCoprocessor(Indexer.class.getName())) {
                 Map<String, String> opts = Maps.newHashMapWithExpectedSize(1);
                 opts.put(CoveredColumnsIndexBuilder.CODEC_CLASS_NAME_KEY, 
PhoenixIndexCodec.class.getName());
@@ -645,7 +644,7 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
                 
descriptor.addCoprocessor(MultiRowMutationEndpoint.class.getName(),
                         null, priority, null);
             }
-            
+
             if 
(descriptor.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES) != null
                     && 
Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(descriptor
                             
.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES)))) {
@@ -708,7 +707,7 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
                     }
                     modifyColumnFamilyDescriptor(newColumnDesc, family);
                 }
-                
+
                 if (newColumnDesc.equals(oldColumnDesc)) {
                     // Table already has family and it's the same.
                     return;
@@ -722,7 +721,7 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
                 sqlE = new 
SQLExceptionInfo.Builder(SQLExceptionCode.INTERRUPTED_EXCEPTION).setRootCause(e).build().buildException();
             } catch (TimeoutException e) {
                 sqlE = new 
SQLExceptionInfo.Builder(SQLExceptionCode.OPERATION_TIMED_OUT).setRootCause(e.getCause()
 != null ? e.getCause() : e).build().buildException();
-            } 
+            }
         } catch (IOException e) {
             sqlE = ServerUtil.parseServerException(e);
         } finally {
@@ -743,7 +742,7 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
             }
         }
     }
-    
+
     private void addOrModifyColumnDescriptor(byte[] tableName, HBaseAdmin 
admin, HColumnDescriptor oldColumnDesc,
             HColumnDescriptor newColumnDesc) throws IOException, 
InterruptedException, TimeoutException {
         boolean isOnlineSchemaUpgradeEnabled = 
ConnectionQueryServicesImpl.this.props.getBoolean(
@@ -766,12 +765,12 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
             pollForUpdatedColumnDescriptor(admin, tableName, newColumnDesc);
         }
     }
-    
+
     private static interface RetriableOperation {
         boolean checkForCompletion() throws TimeoutException, IOException;
         String getOperatioName();
     }
-    
+
     private void pollForUpdatedTableDescriptor(final HBaseAdmin admin, final 
HTableDescriptor newTableDescriptor,
             final byte[] tableName) throws InterruptedException, 
TimeoutException {
         checkAndRetry(new RetriableOperation() {
@@ -788,7 +787,7 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
             }
         });
     }
-    
+
     private void pollForUpdatedColumnDescriptor(final HBaseAdmin admin, final 
byte[] tableName,
             final HColumnDescriptor columnFamilyDesc) throws 
InterruptedException, TimeoutException {
         checkAndRetry(new RetriableOperation() {
@@ -805,7 +804,7 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
             }
         });
     }
-    
+
     private void checkAndRetry(RetriableOperation op) throws 
InterruptedException, TimeoutException {
         int maxRetries = ConnectionQueryServicesImpl.this.props.getInt(
                 QueryServices.NUM_RETRIES_FOR_SCHEMA_UPDATE_CHECK,
@@ -834,9 +833,9 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
             numTries++;
             Thread.sleep(sleepInterval);
         } while (numTries < maxRetries && !success);
-        
+
         watch.stop();
-        
+
         if (!success) {
             throw new TimeoutException("Operation  " + op.getOperatioName() + 
" didn't complete within "
                     + watch.elapsedMillis() + " ms "
@@ -856,11 +855,11 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
     private boolean allowOnlineTableSchemaUpdate() {
         return props.getBoolean(
                 QueryServices.ALLOW_ONLINE_TABLE_SCHEMA_UPDATE,
-                
QueryServicesOptions.DEFAULT_ALLOW_ONLINE_TABLE_SCHEMA_UPDATE);    
+                QueryServicesOptions.DEFAULT_ALLOW_ONLINE_TABLE_SCHEMA_UPDATE);
     }
-    
+
     /**
-     * 
+     *
      * @param tableName
      * @param splits
      * @param modifyExistingMetaData TODO
@@ -890,7 +889,7 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
             }
 
             HTableDescriptor newDesc = generateTableDescriptor(tableName, 
existingDesc, tableType , props, families, splits);
-            
+
             if (!tableExist) {
                 if 
(newDesc.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES) != null && 
Boolean.TRUE.equals(
                     
PBoolean.INSTANCE.toObject(newDesc.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES))))
 {
@@ -933,7 +932,7 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
                 if (isMetaTable) {
                     checkClientServerCompatibility();
                 }
-                         
+
                 if (!modifyExistingMetaData || existingDesc.equals(newDesc)) {
                     return existingDesc;
                 }
@@ -969,7 +968,7 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
         }
         return null; // will never make it here
     }
-    
+
     @Override
     public void modifyTable(byte[] tableName, HTableDescriptor newDesc) throws 
IOException,
     InterruptedException, TimeoutException {
@@ -991,7 +990,7 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
         }
         return 
!MetaDataUtil.decodeMutableIndexConfiguredProperly(serverVersion);
     }
-    
+
     private static boolean isCompatible(Long serverVersion) {
         if (serverVersion == null) {
             return false;
@@ -1025,7 +1024,7 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
                             BlockingRpcCallback<GetVersionResponse> 
rpcCallback =
                                     new 
BlockingRpcCallback<GetVersionResponse>();
                             GetVersionRequest.Builder builder = 
GetVersionRequest.newBuilder();
-                            
+
                             instance.getVersion(controller, builder.build(), 
rpcCallback);
                             if(controller.getFailedOn() != null) {
                                 throw controller.getFailedOn();
@@ -1099,7 +1098,7 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
             throw new SQLException(t);
         }
     }
-    
+
     // Our property values are translated using toString, so we need to 
"string-ify" this.
     private static final String TRUE_BYTES_AS_STRING = 
Bytes.toString(PDataType.TRUE_BYTES);
 
@@ -1109,7 +1108,7 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
             maxFileSize = this.config.getLong(HConstants.HREGION_MAX_FILESIZE, 
HConstants.DEFAULT_MAX_FILE_SIZE);
         }
         byte[] physicalIndexName = 
MetaDataUtil.getViewIndexPhysicalName(physicalTableName);
-        
+
         int indexMaxFileSizePerc;
         // Get percentage to use from table props first and then fallback to 
config
         Integer indexMaxFileSizePercProp = 
(Integer)tableProps.remove(QueryServices.INDEX_MAX_FILESIZE_PERC_ATTRIB);
@@ -1135,7 +1134,7 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
 
     private void ensureLocalIndexTableCreated(byte[] physicalTableName, 
Map<String,Object> tableProps, List<Pair<byte[],Map<String,Object>>> families, 
byte[][] splits, long timestamp) throws SQLException {
         PTable table;
-        String parentTableName = Bytes.toString(physicalTableName, 
MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX_BYTES.length, 
+        String parentTableName = Bytes.toString(physicalTableName, 
MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX_BYTES.length,
             physicalTableName.length - 
MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX_BYTES.length);
         try {
             synchronized (latestMetaDataLock) {
@@ -1196,7 +1195,7 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
                 // Ignore, as we may never have created a view index table
             }
         } catch (IOException e) {
-            throw ServerUtil.parseServerException(e); 
+            throw ServerUtil.parseServerException(e);
         } finally {
             try {
                 if (admin != null) admin.close();
@@ -1231,7 +1230,7 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
                 // Ignore, as we may never have created a view index table
             }
         } catch (IOException e) {
-            throw ServerUtil.parseServerException(e); 
+            throw ServerUtil.parseServerException(e);
         } finally {
             try {
                 if (admin != null) admin.close();
@@ -1293,7 +1292,7 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
             ensureViewIndexTableCreated(tableName, tableProps, 
familiesPlusDefault, MetaDataUtil.isSalted(m, kvBuilder, ptr) ? splits : null, 
MetaDataUtil.getClientTimeStamp(m));
             
ensureLocalIndexTableCreated(MetaDataUtil.getLocalIndexPhysicalName(tableName), 
tableProps, familiesPlusDefault, splits);
         }
-        
+
         byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes, schemaBytes, 
tableBytes);
         MetaDataMutationResult result = metaDataCoprocessorExec(tableKey,
             new Batch.Call<MetaDataService, MetaDataResponse>() {
@@ -1335,7 +1334,7 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
                     
builder.setTableName(HBaseZeroCopyByteString.wrap(tableBytes));
                     builder.setTableTimestamp(tableTimestamp);
                     builder.setClientTimestamp(clientTimestamp);
-                    
+
                    instance.getTable(controller, builder.build(), rpcCallback);
                    if(controller.getFailedOn() != null) {
                        throw controller.getFailedOn();
@@ -1367,7 +1366,7 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
                         }
                         builder.setTableType(tableType.getSerializedValue());
                         builder.setCascade(cascade);
-                        
+
                         instance.dropTable(controller, builder.build(), 
rpcCallback);
                         if(controller.getFailedOn() != null) {
                             throw controller.getFailedOn();
@@ -1375,7 +1374,7 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
                         return rpcCallback.get();
                     }
                 });
-        
+
         final MutationCode code = result.getMutationCode();
         switch(code) {
         case TABLE_ALREADY_EXISTS:
@@ -1398,7 +1397,7 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
         }
           return result;
     }
-    
+
     private void invalidateTables(final List<byte[]> tableNamesToDelete) {
         if (tableNamesToDelete != null) {
             for ( byte[] tableName : tableNamesToDelete ) {
@@ -1406,7 +1405,7 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
             }
         }
     }
-    
+
     private void dropTables(final List<byte[]> tableNamesToDelete) throws 
SQLException {
         HBaseAdmin admin = null;
         SQLException sqlE = null;
@@ -1421,7 +1420,7 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
                     }
                 }
             }
-            
+
         } catch (IOException e) {
             sqlE = ServerUtil.parseServerException(e);
         } finally {
@@ -1452,11 +1451,11 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
         }
         return props;
     }
-    
+
     private void ensureViewIndexTableCreated(PName tenantId, byte[] 
physicalIndexTableName, long timestamp) throws SQLException {
         PTable table;
         String name = Bytes.toString(
-                physicalIndexTableName, 
+                physicalIndexTableName,
                 MetaDataUtil.VIEW_INDEX_TABLE_PREFIX_BYTES.length,
                 
physicalIndexTableName.length-MetaDataUtil.VIEW_INDEX_TABLE_PREFIX_BYTES.length);
         try {
@@ -1479,7 +1478,7 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
         }
         ensureViewIndexTableCreated(table, timestamp);
     }
-    
+
     private void ensureViewIndexTableCreated(PTable table, long timestamp) 
throws SQLException {
         byte[] physicalTableName = table.getPhysicalName().getBytes();
         HTableDescriptor htableDesc = 
this.getTableDescriptor(physicalTableName);
@@ -1503,10 +1502,10 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
         if (table.getBucketNum() != null) {
             splits = SaltingUtil.getSalteByteSplitPoints(table.getBucketNum());
         }
-        
+
         ensureViewIndexTableCreated(physicalTableName, tableProps, families, 
splits, timestamp);
     }
-    
+
     @Override
     public MetaDataMutationResult addColumn(final List<Mutation> 
tableMetaData, List<Pair<byte[],Map<String,Object>>> families, PTable table) 
throws SQLException {
         byte[][] rowKeyMetaData = new byte[3][];
@@ -1543,7 +1542,7 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
                         MutationProto mp = ProtobufUtil.toProto(m);
                         builder.addTableMetadataMutations(mp.toByteString());
                     }
-                    
+
                     instance.addColumn(controller, builder.build(), 
rpcCallback);
                     if(controller.getFailedOn() != null) {
                         throw controller.getFailedOn();
@@ -1558,7 +1557,7 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
                && Boolean.FALSE.equals(PBoolean.INSTANCE.toObject(ptr))) {
                 flushTable(table.getPhysicalName().getBytes());
             }
-            
+
             if (tableType == PTableType.TABLE) {
                 // If we're changing MULTI_TENANT to true or false, create or 
drop the view index table
                 if (MetaDataUtil.getMutationValue(m, 
PhoenixDatabaseMetaData.MULTI_TENANT_BYTES, kvBuilder, ptr)){
@@ -1615,11 +1614,11 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
             break;
         }
         return result;
-       
+
     }
 
     // Keeping this to use for further upgrades
-    protected PhoenixConnection addColumnsIfNotExists(PhoenixConnection 
oldMetaConnection, 
+    protected PhoenixConnection addColumnsIfNotExists(PhoenixConnection 
oldMetaConnection,
         String tableName, long timestamp, String columns) throws SQLException {
 
         Properties props = new Properties(oldMetaConnection.getClientInfo());
@@ -1648,7 +1647,7 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
         }
         return metaConnection;
     }
-    
+
     @Override
     public void init(final String url, final Properties props) throws 
SQLException {
         try {
@@ -1708,7 +1707,7 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
                                             
PhoenixDatabaseMetaData.SEQUENCE_SCHEMA_NAME,
                                             
PhoenixDatabaseMetaData.SEQUENCE_TABLE_NAME,
                                             
MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP);
-                                    
clearTableFromCache(ByteUtil.EMPTY_BYTE_ARRAY, 
+                                    
clearTableFromCache(ByteUtil.EMPTY_BYTE_ARRAY,
                                             
PhoenixDatabaseMetaData.SEQUENCE_SCHEMA_NAME_BYTES,
                                             
PhoenixDatabaseMetaData.SEQUENCE_TABLE_NAME_BYTES,
                                             
MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP);
@@ -1801,7 +1800,7 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
                         if(controller.getFailedOn() != null) {
                             throw controller.getFailedOn();
                         }
-                        return rpcCallback.get(); 
+                        return rpcCallback.get();
                     }
                   });
             } catch (IOException e) {
@@ -1958,7 +1957,7 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
             sequence.getLock().unlock();
         }
     }
-    
+
     /**
      * Verifies that sequences exist and reserves values for them if 
reserveValues is true
      */
@@ -1966,15 +1965,15 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
     public void validateSequences(List<SequenceKey> sequenceKeys, long 
timestamp, long[] values, SQLException[] exceptions, Sequence.ValueOp action) 
throws SQLException {
         incrementSequenceValues(sequenceKeys, timestamp, values, exceptions, 
action);
     }
-    
+
     /**
      * Increment any of the set of sequences that need more values. These are 
the sequences
      * that are asking for the next value within a given statement. The 
returned sequences
-     * are the ones that were not found because they were deleted by another 
client. 
+     * are the ones that were not found because they were deleted by another 
client.
      * @param sequenceKeys sorted list of sequence kyes
      * @param timestamp
      * @throws SQLException if any of the sequences cannot be found
-     * 
+     *
      */
     @Override
     public void incrementSequences(List<SequenceKey> sequenceKeys, long 
timestamp, long[] values, SQLException[] exceptions) throws SQLException {
@@ -2057,7 +2056,7 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
             }
         }
     }
-    
+
     @Override
     public void clearTableFromCache(final byte[] tenantId, final byte[] 
schemaName, final byte[] tableName,
             final long clientTS) throws SQLException {
@@ -2216,7 +2215,7 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
             }
         }
     }
-    
+
     @Override
     public void addConnection(PhoenixConnection connection) throws 
SQLException {
         synchronized (connectionCountLock) {
@@ -2254,28 +2253,28 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
         // For now, only Feature is REVERSE_SCAN and it's not supported in any 
version yet
         return false;
     }
-    
+
     @Override
     public String getUserName() {
         return userName;
     }
-    
+
     private void checkClosed() {
         if (closed) {
             throwConnectionClosedException();
         }
     }
-    
+
     private void throwConnectionClosedIfNullMetaData() {
         if (latestMetaData == null) {
             throwConnectionClosedException();
         }
     }
-    
+
     private void throwConnectionClosedException() {
         throw new IllegalStateException("Connection to the cluster is closed");
     }
-    
+
     @Override
     public PTableStats getTableStats(final byte[] physicalName, final long 
clientTimeStamp) throws SQLException {
         try {
@@ -2304,13 +2303,13 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
                         }
                     }
                 }
-                
+
             });
         } catch (ExecutionException e) {
             throw ServerUtil.parseServerException(e);
         }
     }
-    
+
     @Override
     public int getSequenceSaltBuckets() {
         return nSequenceSaltBuckets;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c5edd3a7/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
----------------------------------------------------------------------
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java 
b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
index 93dfc5a..ce9016d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
@@ -29,12 +29,12 @@ import org.apache.phoenix.util.SQLCloseable;
 
 
 /**
- * 
+ *
  * Interface to group together services needed during querying.  The
  * parameters that may be set in {@link org.apache.hadoop.conf.Configuration}
  * are documented here: https://github.com/forcedotcom/phoenix/wiki/Tuning
- *     
- * 
+ *
+ *
  * @since 0.1
  */
 @Immutable
@@ -48,7 +48,7 @@ public interface QueryServices extends SQLCloseable {
     public static final String HBASE_CLIENT_PRINCIPAL = 
"hbase.myclient.principal";
     public static final String SPOOL_DIRECTORY = "phoenix.spool.directory";
     public static final String AUTO_COMMIT_ATTRIB = 
"phoenix.connection.autoCommit";
-    
+
     /**
         * max size to spool the the result into
         * ${java.io.tmpdir}/ResultSpoolerXXX.bin if
@@ -56,10 +56,10 @@ public interface QueryServices extends SQLCloseable {
         * <p>
         * default is unlimited(-1)
         * <p>
-        * if the threshold is reached, a {@link SpoolTooBigToDiskException } 
will be thrown 
+        * if the threshold is reached, a {@link SpoolTooBigToDiskException } 
will be thrown
         */
        public static final String MAX_SPOOL_TO_DISK_BYTES_ATTRIB = 
"phoenix.query.maxSpoolToDiskBytes";
-    
+
     /**
      * Number of records to read per chunk when streaming records of a basic 
scan.
      */
@@ -88,7 +88,7 @@ public interface QueryServices extends SQLCloseable {
     public static final String GROUPBY_ESTIMATED_DISTINCT_VALUES_ATTRIB = 
"phoenix.groupby.estimatedDistinctValues";
 
     public static final String CALL_QUEUE_PRODUCER_ATTRIB_NAME = 
"CALL_QUEUE_PRODUCER";
-    
+
     public static final String MASTER_INFO_PORT_ATTRIB = 
"hbase.master.info.port";
     public static final String REGIONSERVER_INFO_PORT_ATTRIB = 
"hbase.regionserver.info.port";
     public static final String REGIONSERVER_LEASE_PERIOD_ATTRIB = 
"hbase.regionserver.lease.period";
@@ -106,19 +106,19 @@ public interface QueryServices extends SQLCloseable {
     public static final String AUTO_UPGRADE_WHITELIST_ATTRIB = 
"phoenix.client.autoUpgradeWhiteList";
     // Mainly for testing to force spilling
     public static final String MAX_MEMORY_SIZE_ATTRIB = 
"phoenix.query.maxGlobalMemorySize";
-    
+
     // The following config settings is to deal with SYSTEM.CATALOG 
moves(PHOENIX-916) among region servers
     public static final String CLOCK_SKEW_INTERVAL_ATTRIB = 
"phoenix.clock.skew.interval";
-    
-    // A master switch if to enable auto rebuild an index which failed to be 
updated previously 
+
+    // A master switch if to enable auto rebuild an index which failed to be 
updated previously
     public static final String INDEX_FAILURE_HANDLING_REBUILD_ATTRIB = 
"phoenix.index.failure.handling.rebuild";
-    
+
     // Time interval to check if there is an index needs to be rebuild
-    public static final String INDEX_FAILURE_HANDLING_REBUILD_INTERVAL_ATTRIB 
= 
+    public static final String INDEX_FAILURE_HANDLING_REBUILD_INTERVAL_ATTRIB =
         "phoenix.index.failure.handling.rebuild.interval";
-    
-    // Index will be partially re-built from index disable time stamp - 
following overlap time 
-    public static final String 
INDEX_FAILURE_HANDLING_REBUILD_OVERLAP_TIME_ATTRIB = 
+
+    // Index will be partially re-built from index disable time stamp - 
following overlap time
+    public static final String 
INDEX_FAILURE_HANDLING_REBUILD_OVERLAP_TIME_ATTRIB =
         "phoenix.index.failure.handling.rebuild.overlap.time";
     public static final String MIN_INDEX_PRIOIRTY_ATTRIB = 
"phoenix.regionserver.index.priority.min";
     public static final String MAX_INDEX_PRIOIRTY_ATTRIB = 
"phoenix.regionserver.index.priority.max";
@@ -147,7 +147,8 @@ public interface QueryServices extends SQLCloseable {
     public static final String NUM_RETRIES_FOR_SCHEMA_UPDATE_CHECK = 
"phoenix.schema.change.retries";
     public static final String DELAY_FOR_SCHEMA_UPDATE_CHECK = 
"phoenix.schema.change.delay";
     public static final String DEFAULT_KEEP_DELETED_CELLS_ATTRIB = 
"phoenix.table.default.keep.deleted.cells";
-    
+    public static final String DEFAULT_STORE_NULLS_ATTRIB = 
"phoenix.table.default.store.nulls";
+
     /**
      * Get executor service used for parallel scans
      */
@@ -156,13 +157,13 @@ public interface QueryServices extends SQLCloseable {
      * Get the memory manager used to track memory usage
      */
     public MemoryManager getMemoryManager();
-    
+
     /**
      * Get the properties from the HBase configuration in a
      * read-only structure that avoids any synchronization
      */
     public ReadOnlyProps getProps();
-    
+
     /**
      * Get query optimizer used to choose the best query plan
      */

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c5edd3a7/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
----------------------------------------------------------------------
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java 
b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
index d53c70d..5913796 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
@@ -75,8 +75,8 @@ import org.apache.phoenix.util.ReadOnlyProps;
 
 /**
  * Options for {@link QueryServices}.
- * 
- * 
+ *
+ *
  * @since 0.1
  */
 public class QueryServicesOptions {
@@ -94,13 +94,13 @@ public class QueryServicesOptions {
     public static final int DEFAULT_MAX_QUERY_CONCURRENCY = 64;
     public static final String DEFAULT_DATE_FORMAT = 
DateUtil.DEFAULT_DATE_FORMAT;
     public static final String DEFAULT_DATE_FORMAT_TIMEZONE = 
DateUtil.DEFAULT_TIME_ZONE_ID;
-    public static final boolean DEFAULT_CALL_QUEUE_ROUND_ROBIN = true; 
+    public static final boolean DEFAULT_CALL_QUEUE_ROUND_ROBIN = true;
     public static final int DEFAULT_MAX_MUTATION_SIZE = 500000;
     public static final boolean DEFAULT_ROW_KEY_ORDER_SALTED_TABLE = true; // 
Merge sort on client to ensure salted tables are row key ordered
     public static final boolean DEFAULT_USE_INDEXES = true; // Use indexes
     public static final boolean DEFAULT_IMMUTABLE_ROWS = false; // Tables rows 
may be updated
     public static final boolean DEFAULT_DROP_METADATA = true; // Drop meta 
data also.
-    
+
     public final static int DEFAULT_MUTATE_BATCH_SIZE = 1000; // Batch size 
for UPSERT SELECT and DELETE
        // The only downside of it being out-of-sync is that the 
parallelization of the scan won't be as balanced as it could be.
     public static final int DEFAULT_MAX_SERVER_CACHE_TIME_TO_LIVE_MS = 30000; 
// 30 sec (with no activity)
@@ -114,8 +114,8 @@ public class QueryServicesOptions {
     // latency and client-side spooling/buffering. Smaller means less initial
     // latency and less parallelization.
     public static final long DEFAULT_SCAN_RESULT_CHUNK_SIZE = 2999;
-    
-    // 
+
+    //
     // Spillable GroupBy - SPGBY prefix
     //
     // Enable / disable spillable group by
@@ -125,10 +125,10 @@ public class QueryServicesOptions {
     public static final int DEFAULT_GROUPBY_SPILL_FILES = 2;
     // Max size of 1st level main memory cache in bytes --> upper bound
     public static final long DEFAULT_GROUPBY_MAX_CACHE_MAX = 1024L*1024L*100L; 
 // 100 Mb
-    
+
     public static final long DEFAULT_SEQUENCE_CACHE_SIZE = 100;  // reserve 
100 sequences at a time
     public static final int DEFAULT_INDEX_MAX_FILESIZE_PERC = 50; // % of data 
table max file size for index table
-    public static final long DEFAULT_MAX_SERVER_METADATA_CACHE_TIME_TO_LIVE_MS 
=  60000 * 30; // 30 mins   
+    public static final long DEFAULT_MAX_SERVER_METADATA_CACHE_TIME_TO_LIVE_MS 
=  60000 * 30; // 30 mins
     public static final long DEFAULT_MAX_SERVER_METADATA_CACHE_SIZE =  
1024L*1024L*20L; // 20 Mb
     public static final long DEFAULT_MAX_CLIENT_METADATA_CACHE_SIZE =  
1024L*1024L*10L; // 10 Mb
     public static final int DEFAULT_GROUPBY_ESTIMATED_DISTINCT_VALUES = 1000;
@@ -136,7 +136,7 @@ public class QueryServicesOptions {
     public static final boolean DEFAULT_INDEX_FAILURE_HANDLING_REBUILD = true; 
// auto rebuild on
     public static final long DEFAULT_INDEX_FAILURE_HANDLING_REBUILD_INTERVAL = 
10000; // 10 secs
     public static final long 
DEFAULT_INDEX_FAILURE_HANDLING_REBUILD_OVERLAP_TIME = 300000; // 5 mins
-    
+
     public static final int DEFAULT_INDEX_MAX_PRIORITY = 1050;
     /**
      * HConstants#HIGH_QOS is the max we will see to a standard table. We go 
higher to differentiate
@@ -144,7 +144,7 @@ public class QueryServicesOptions {
      */
     public static final int DEFAULT_INDEX_MIN_PRIORITY = 1000;
     public static final int DEFAULT_INDEX_HANDLER_COUNT = 30;
-    
+
     public static final int DEFAULT_TRACING_PAGE_SIZE = 100;
     /**
      * Configuration key to overwrite the tablename that should be used as the 
target table
@@ -162,7 +162,7 @@ public class QueryServicesOptions {
     public static final boolean DEFAULT_STATS_USE_CURRENT_TIME = true;
 
     public static final boolean DEFAULT_USE_REVERSE_SCAN = true;
-    
+
     /**
      * Use only first time SYSTEM.SEQUENCE table is created.
      */
@@ -176,20 +176,21 @@ public class QueryServicesOptions {
     public static final int DEFAULT_RETRIES_FOR_SCHEMA_UPDATE_CHECK = 10;
     public static final long DEFAULT_DELAY_FOR_SCHEMA_UPDATE_CHECK = 5 * 1000; 
// 5 seconds.
     public static final boolean DEFAULT_KEEP_DELETED_CELLS = false;
+    public static final boolean DEFAULT_STORE_NULLS = false;
 
     // TODO Change this to true as part of PHOENIX-1543
     public static final boolean DEFAULT_AUTO_COMMIT = false;
-    
+
     private final Configuration config;
 
     private QueryServicesOptions(Configuration config) {
         this.config = config;
     }
-    
+
     public ReadOnlyProps getProps(ReadOnlyProps defaultProps) {
         return new ReadOnlyProps(defaultProps, config.iterator());
     }
-    
+
     public QueryServicesOptions setAll(ReadOnlyProps props) {
         for (Entry<String,String> entry : props) {
             config.set(entry.getKey(), entry.getValue());
@@ -243,7 +244,7 @@ public class QueryServicesOptions {
         }
         return options;
     }
-    
+
     public Configuration getConfiguration() {
         return config;
     }
@@ -252,12 +253,12 @@ public class QueryServicesOptions {
         config.setIfUnset(name, Integer.toString(value));
         return this;
     }
-    
+
     private QueryServicesOptions setIfUnset(String name, boolean value) {
         config.setIfUnset(name, Boolean.toString(value));
         return this;
     }
-    
+
     private QueryServicesOptions setIfUnset(String name, long value) {
         config.setIfUnset(name, Long.toString(value));
         return this;
@@ -267,24 +268,24 @@ public class QueryServicesOptions {
         config.setIfUnset(name, value);
         return this;
     }
-    
+
     public QueryServicesOptions setKeepAliveMs(int keepAliveMs) {
         return set(KEEP_ALIVE_MS_ATTRIB, keepAliveMs);
     }
-    
+
     public QueryServicesOptions setThreadPoolSize(int threadPoolSize) {
         return set(THREAD_POOL_SIZE_ATTRIB, threadPoolSize);
     }
-    
+
     public QueryServicesOptions setQueueSize(int queueSize) {
         config.setInt(QUEUE_SIZE_ATTRIB, queueSize);
         return this;
     }
-    
+
     public QueryServicesOptions setThreadTimeoutMs(int threadTimeoutMs) {
         return set(THREAD_TIMEOUT_MS_ATTRIB, threadTimeoutMs);
     }
-    
+
     public QueryServicesOptions setSpoolThresholdBytes(int 
spoolThresholdBytes) {
         return set(SPOOL_THRESHOLD_BYTES_ATTRIB, spoolThresholdBytes);
     }
@@ -296,15 +297,15 @@ public class QueryServicesOptions {
     public QueryServicesOptions setMaxMemoryPerc(int maxMemoryPerc) {
         return set(MAX_MEMORY_PERC_ATTRIB, maxMemoryPerc);
     }
-    
+
     public QueryServicesOptions setMaxMemoryWaitMs(int maxMemoryWaitMs) {
         return set(MAX_MEMORY_WAIT_MS_ATTRIB, maxMemoryWaitMs);
     }
-    
+
     public QueryServicesOptions setMaxTenantMemoryPerc(int 
maxTenantMemoryPerc) {
         return set(MAX_TENANT_MEMORY_PERC_ATTRIB, maxTenantMemoryPerc);
     }
-    
+
     public QueryServicesOptions setMaxServerCacheSize(long maxServerCacheSize) 
{
         return set(MAX_SERVER_CACHE_SIZE_ATTRIB, maxServerCacheSize);
     }
@@ -320,31 +321,31 @@ public class QueryServicesOptions {
     public QueryServicesOptions setScanFetchSize(int scanFetchSize) {
         return set(SCAN_CACHE_SIZE_ATTRIB, scanFetchSize);
     }
-    
+
     public QueryServicesOptions setDateFormat(String dateFormat) {
         return set(DATE_FORMAT_ATTRIB, dateFormat);
     }
-    
+
     public QueryServicesOptions setCallQueueRoundRobin(boolean isRoundRobin) {
         return set(CALL_QUEUE_PRODUCER_ATTRIB_NAME, isRoundRobin);
     }
-    
+
     public QueryServicesOptions setMaxMutateSize(int maxMutateSize) {
         return set(MAX_MUTATION_SIZE_ATTRIB, maxMutateSize);
     }
-    
+
     public QueryServicesOptions setMutateBatchSize(int mutateBatchSize) {
         return set(MUTATE_BATCH_SIZE_ATTRIB, mutateBatchSize);
     }
-    
+
     public QueryServicesOptions setRowKeyOrderSaltedTable(boolean 
rowKeyOrderSaltedTable) {
         return set(ROW_KEY_ORDER_SALTED_TABLE_ATTRIB, rowKeyOrderSaltedTable);
     }
-    
+
     public QueryServicesOptions setDropMetaData(boolean dropMetadata) {
         return set(DROP_METADATA_ATTRIB, dropMetadata);
     }
-    
+
     public QueryServicesOptions setGroupBySpill(boolean enabled) {
         return set(GROUPBY_SPILLABLE_ATTRIB, enabled);
     }
@@ -352,27 +353,27 @@ public class QueryServicesOptions {
     public QueryServicesOptions setGroupBySpillMaxCacheSize(long size) {
         return set(GROUPBY_MAX_CACHE_SIZE_ATTRIB, size);
     }
-    
+
     public QueryServicesOptions setGroupBySpillNumSpillFiles(long num) {
         return set(GROUPBY_SPILL_FILES_ATTRIB, num);
     }
 
-    
+
     private QueryServicesOptions set(String name, boolean value) {
         config.set(name, Boolean.toString(value));
         return this;
     }
-    
+
     private QueryServicesOptions set(String name, int value) {
         config.set(name, Integer.toString(value));
         return this;
     }
-    
+
     private QueryServicesOptions set(String name, String value) {
         config.set(name, value);
         return this;
     }
-    
+
     private QueryServicesOptions set(String name, long value) {
         config.set(name, Long.toString(value));
         return this;
@@ -381,19 +382,19 @@ public class QueryServicesOptions {
     public int getKeepAliveMs() {
         return config.getInt(KEEP_ALIVE_MS_ATTRIB, DEFAULT_KEEP_ALIVE_MS);
     }
-    
+
     public int getThreadPoolSize() {
         return config.getInt(THREAD_POOL_SIZE_ATTRIB, 
DEFAULT_THREAD_POOL_SIZE);
     }
-    
+
     public int getQueueSize() {
         return config.getInt(QUEUE_SIZE_ATTRIB, DEFAULT_QUEUE_SIZE);
     }
-    
+
     public int getMaxMemoryPerc() {
         return config.getInt(MAX_MEMORY_PERC_ATTRIB, DEFAULT_MAX_MEMORY_PERC);
     }
-    
+
     public int getMaxMemoryWaitMs() {
         return config.getInt(MAX_MEMORY_WAIT_MS_ATTRIB, 
DEFAULT_MAX_MEMORY_WAIT_MS);
     }
@@ -413,47 +414,47 @@ public class QueryServicesOptions {
     public boolean isImmutableRows() {
         return config.getBoolean(IMMUTABLE_ROWS_ATTRIB, 
DEFAULT_IMMUTABLE_ROWS);
     }
-    
+
     public boolean isDropMetaData() {
         return config.getBoolean(DROP_METADATA_ATTRIB, DEFAULT_DROP_METADATA);
     }
-    
+
     public boolean isSpillableGroupByEnabled() {
         return config.getBoolean(GROUPBY_SPILLABLE_ATTRIB, 
DEFAULT_GROUPBY_SPILLABLE);
     }
-    
+
     public long getSpillableGroupByMaxCacheSize() {
         return config.getLong(GROUPBY_MAX_CACHE_SIZE_ATTRIB, 
DEFAULT_GROUPBY_MAX_CACHE_MAX);
     }
-    
+
     public int getSpillableGroupByNumSpillFiles() {
         return config.getInt(GROUPBY_SPILL_FILES_ATTRIB, 
DEFAULT_GROUPBY_SPILL_FILES);
     }
-    
+
     public QueryServicesOptions setMaxServerCacheTTLMs(int ttl) {
         return set(MAX_SERVER_CACHE_TIME_TO_LIVE_MS_ATTRIB, ttl);
     }
-    
+
     public QueryServicesOptions setMasterInfoPort(int port) {
         return set(MASTER_INFO_PORT_ATTRIB, port);
     }
-    
+
     public QueryServicesOptions setRegionServerInfoPort(int port) {
         return set(REGIONSERVER_INFO_PORT_ATTRIB, port);
     }
-    
+
     public QueryServicesOptions setRegionServerLeasePeriodMs(int period) {
         return set(REGIONSERVER_LEASE_PERIOD_ATTRIB, period);
     }
-    
+
     public QueryServicesOptions setRpcTimeoutMs(int timeout) {
         return set(RPC_TIMEOUT_ATTRIB, timeout);
     }
-    
+
     public QueryServicesOptions setUseIndexes(boolean useIndexes) {
         return set(USE_INDEXES_ATTRIB, useIndexes);
     }
-    
+
     public QueryServicesOptions setImmutableRows(boolean isImmutableRows) {
         return set(IMMUTABLE_ROWS_ATTRIB, isImmutableRows);
     }
@@ -469,34 +470,34 @@ public class QueryServicesOptions {
     public QueryServicesOptions setStatsUpdateFrequencyMs(int frequencyMs) {
         return set(STATS_UPDATE_FREQ_MS_ATTRIB, frequencyMs);
     }
-    
+
     public QueryServicesOptions setMinStatsUpdateFrequencyMs(int frequencyMs) {
         return set(MIN_STATS_UPDATE_FREQ_MS_ATTRIB, frequencyMs);
-    }    
-    
+    }
+
     public QueryServicesOptions setSequenceSaltBuckets(int saltBuckets) {
         config.setInt(SEQUENCE_SALT_BUCKETS_ATTRIB, saltBuckets);
         return this;
     }
-    
+
     public QueryServicesOptions setExplainChunkCount(boolean showChunkCount) {
         config.setBoolean(EXPLAIN_CHUNK_COUNT_ATTRIB, showChunkCount);
         return this;
     }
-    
+
     public QueryServicesOptions setAllowOnlineSchemaUpdate(boolean allow) {
         config.setBoolean(ALLOW_ONLINE_TABLE_SCHEMA_UPDATE, allow);
         return this;
     }
-    
+
     public QueryServicesOptions setNumRetriesForSchemaChangeCheck(int 
numRetries) {
         config.setInt(NUM_RETRIES_FOR_SCHEMA_UPDATE_CHECK, numRetries);
         return this;
     }
-    
+
     public QueryServicesOptions setDelayInMillisForSchemaChangeCheck(long 
delayInMillis) {
         config.setLong(DELAY_FOR_SCHEMA_UPDATE_CHECK, delayInMillis);
         return this;
     }
-    
+
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c5edd3a7/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
----------------------------------------------------------------------
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 71d5827..ea3af37 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
@@ -1251,9 +1251,12 @@ public class MetaDataClient {
                 disableWAL = disableWALProp;
             }
 
-            Boolean storeNullsProp = (Boolean) 
tableProps.remove(PhoenixDatabaseMetaData
-                    .STORE_NULLS);
-            storeNulls = storeNullsProp == null ? false : storeNullsProp;
+            Boolean storeNullsProp = (Boolean) 
tableProps.remove(PhoenixDatabaseMetaData.STORE_NULLS);
+            storeNulls = storeNullsProp == null
+                    ? connection.getQueryServices().getProps().getBoolean(
+                            QueryServices.DEFAULT_STORE_NULLS_ATTRIB,
+                            QueryServicesOptions.DEFAULT_STORE_NULLS)
+                    : storeNullsProp;
 
             // Delay this check as it is supported to have IMMUTABLE_ROWS and 
SALT_BUCKETS defined on views
             if ((statement.getTableType() == PTableType.VIEW || indexId != 
null) && !tableProps.isEmpty()) {

Reply via email to