You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ma...@apache.org on 2016/02/02 04:59:35 UTC

[01/50] [abbrv] phoenix git commit: PHOENIX-2520 Create DDL property for metadata update frequency

Repository: phoenix
Updated Branches:
  refs/heads/calcite 518356479 -> 74409be82


http://git-wip-us.apache.org/repos/asf/phoenix/blob/59b336ec/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
----------------------------------------------------------------------
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 1805d94..1611466 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
@@ -135,6 +135,7 @@ public class PTableImpl implements PTable {
     private boolean rowKeyOrderOptimizable; // TODO: remove when required that tables have been upgrade for PHOENIX-2067
     private boolean hasColumnsRequiringUpgrade; // TODO: remove when required that tables have been upgrade for PHOENIX-2067
     private int rowTimestampColPos;
+    private long updateCacheFrequency;
 
     public PTableImpl() {
         this.indexes = Collections.emptyList();
@@ -205,7 +206,7 @@ public class PTableImpl implements PTable {
                 table.getSequenceNumber(), table.getPKName(), table.getBucketNum(), getColumnsToClone(table), parentSchemaName, table.getParentTableName(),
                 indexes, table.isImmutableRows(), table.getPhysicalNames(), table.getDefaultFamilyName(), viewStatement,
                 table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), table.getIndexType(),
-                table.getTableStats(), table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.isTransactional());
+                table.getTableStats(), table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.isTransactional(), table.getUpdateCacheFrequency());
     }
 
     public static PTableImpl makePTable(PTable table, List<PColumn> columns) throws SQLException {
@@ -214,7 +215,7 @@ public class PTableImpl implements PTable {
                 table.getSequenceNumber(), table.getPKName(), table.getBucketNum(), columns, table.getParentSchemaName(), table.getParentTableName(),
                 table.getIndexes(), table.isImmutableRows(), table.getPhysicalNames(), table.getDefaultFamilyName(), table.getViewStatement(),
                 table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), table.getIndexType(),
-                table.getTableStats(), table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.isTransactional());
+                table.getTableStats(), table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.isTransactional(), table.getUpdateCacheFrequency());
     }
 
     public static PTableImpl makePTable(PTable table, long timeStamp, long sequenceNumber, List<PColumn> columns) throws SQLException {
@@ -223,7 +224,7 @@ public class PTableImpl implements PTable {
                 sequenceNumber, table.getPKName(), table.getBucketNum(), columns, table.getParentSchemaName(), table.getParentTableName(), table.getIndexes(),
                 table.isImmutableRows(), table.getPhysicalNames(), table.getDefaultFamilyName(), table.getViewStatement(), table.isWALDisabled(),
                 table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), table.getIndexType(), table.getTableStats(),
-                table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.isTransactional());
+                table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.isTransactional(), table.getUpdateCacheFrequency());
     }
 
     public static PTableImpl makePTable(PTable table, long timeStamp, long sequenceNumber, List<PColumn> columns, boolean isImmutableRows) throws SQLException {
@@ -232,16 +233,17 @@ public class PTableImpl implements PTable {
                 sequenceNumber, table.getPKName(), table.getBucketNum(), columns, table.getParentSchemaName(), table.getParentTableName(),
                 table.getIndexes(), isImmutableRows, table.getPhysicalNames(), table.getDefaultFamilyName(), table.getViewStatement(),
                 table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(),
-                table.getIndexType(), table.getTableStats(), table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.isTransactional());
+                table.getIndexType(), table.getTableStats(), table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.isTransactional(), table.getUpdateCacheFrequency());
     }
     
-    public static PTableImpl makePTable(PTable table, long timeStamp, long sequenceNumber, List<PColumn> columns, boolean isImmutableRows, boolean isWalDisabled, boolean isMultitenant, boolean storeNulls, boolean isTransactional) throws SQLException {
+    public static PTableImpl makePTable(PTable table, long timeStamp, long sequenceNumber, List<PColumn> columns, boolean isImmutableRows, boolean isWalDisabled,
+            boolean isMultitenant, boolean storeNulls, boolean isTransactional, long updateCacheFrequency) throws SQLException {
         return new PTableImpl(
                 table.getTenantId(), table.getSchemaName(), table.getTableName(), table.getType(), table.getIndexState(), timeStamp,
                 sequenceNumber, table.getPKName(), table.getBucketNum(), columns, table.getParentSchemaName(), table.getParentTableName(),
                 table.getIndexes(), isImmutableRows, table.getPhysicalNames(), table.getDefaultFamilyName(), table.getViewStatement(),
                 isWalDisabled, isMultitenant, storeNulls, table.getViewType(), table.getViewIndexId(), table.getIndexType(), table.getTableStats(),
-                table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), isTransactional);
+                table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), isTransactional, updateCacheFrequency);
     }
     
     public static PTableImpl makePTable(PTable table, PIndexState state) throws SQLException {
@@ -251,7 +253,7 @@ public class PTableImpl implements PTable {
                 table.getParentSchemaName(), table.getParentTableName(), table.getIndexes(),
                 table.isImmutableRows(), table.getPhysicalNames(), table.getDefaultFamilyName(), table.getViewStatement(),
                 table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), table.getIndexType(),
-                table.getTableStats(), table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.isTransactional());
+                table.getTableStats(), table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.isTransactional(), table.getUpdateCacheFrequency());
     }
 
     public static PTableImpl makePTable(PTable table, boolean rowKeyOrderOptimizable) throws SQLException {
@@ -261,7 +263,7 @@ public class PTableImpl implements PTable {
                 table.getParentSchemaName(), table.getParentTableName(), table.getIndexes(),
                 table.isImmutableRows(), table.getPhysicalNames(), table.getDefaultFamilyName(), table.getViewStatement(),
                 table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), table.getIndexType(), table.getTableStats(),
-                table.getBaseColumnCount(), rowKeyOrderOptimizable, table.isTransactional());
+                table.getBaseColumnCount(), rowKeyOrderOptimizable, table.isTransactional(), table.getUpdateCacheFrequency());
     }
 
     public static PTableImpl makePTable(PTable table, PTableStats stats) throws SQLException {
@@ -271,28 +273,33 @@ public class PTableImpl implements PTable {
                 table.getParentSchemaName(), table.getParentTableName(), table.getIndexes(),
                 table.isImmutableRows(), table.getPhysicalNames(), table.getDefaultFamilyName(), table.getViewStatement(),
                 table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), table.getIndexType(), stats,
-                table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.isTransactional());
+                table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.isTransactional(), table.getUpdateCacheFrequency());
     }
 
-    public static PTableImpl makePTable(PName tenantId, PName schemaName, PName tableName, PTableType type, PIndexState state, long timeStamp, long sequenceNumber,
-            PName pkName, Integer bucketNum, List<PColumn> columns, PName dataSchemaName, PName dataTableName, List<PTable> indexes,
-            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) throws SQLException {
+    public static PTableImpl makePTable(PName tenantId, PName schemaName, PName tableName, PTableType type,
+            PIndexState state, long timeStamp, long sequenceNumber, PName pkName, Integer bucketNum,
+            List<PColumn> columns, PName dataSchemaName, PName dataTableName, List<PTable> indexes,
+            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) 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,
-                indexType, PTableStats.EMPTY_STATS, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, rowKeyOrderOptimizable, isTransactional);
+                indexType, PTableStats.EMPTY_STATS, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, rowKeyOrderOptimizable, isTransactional, updateCacheFrequency);
     }
 
     public static PTableImpl makePTable(PName tenantId, PName schemaName, PName tableName, PTableType type,
             PIndexState state, long timeStamp, long sequenceNumber, PName pkName, Integer bucketNum,
             List<PColumn> columns, PName dataSchemaName, PName dataTableName, List<PTable> indexes,
             boolean isImmutableRows, List<PName> physicalNames, PName defaultFamilyName, String viewExpression,
-            boolean disableWAL, boolean multiTenant, boolean storeNulls, ViewType viewType, Short viewIndexId, IndexType indexType, @NotNull PTableStats stats, int baseColumnCount, boolean rowKeyOrderOptimizable, boolean isTransactional)
+            boolean disableWAL, boolean multiTenant, boolean storeNulls, ViewType viewType, Short viewIndexId,
+            IndexType indexType, boolean rowKeyOrderOptimizable, boolean isTransactional, long updateCacheFrequency,
+            @NotNull PTableStats stats, int baseColumnCount)
             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, indexType, stats, baseColumnCount, rowKeyOrderOptimizable, isTransactional);
+                defaultFamilyName, viewExpression, disableWAL, multiTenant, storeNulls, viewType, viewIndexId,
+                indexType, stats, baseColumnCount, rowKeyOrderOptimizable, isTransactional, updateCacheFrequency);
     }
 
     private PTableImpl(PName tenantId, PName schemaName, PName tableName, PTableType type, PIndexState state,
@@ -300,13 +307,19 @@ public class PTableImpl implements PTable {
             PName parentSchemaName, PName parentTableName, List<PTable> indexes, boolean isImmutableRows,
             List<PName> physicalNames, PName defaultFamilyName, String viewExpression, boolean disableWAL, boolean multiTenant,
             boolean storeNulls, ViewType viewType, Short viewIndexId, IndexType indexType,
-            PTableStats stats, int baseColumnCount, boolean rowKeyOrderOptimizable, boolean isTransactional) throws SQLException {
+            PTableStats stats, int baseColumnCount, boolean rowKeyOrderOptimizable, boolean isTransactional, long updateCacheFrequency) throws SQLException {
         init(tenantId, schemaName, tableName, type, state, timeStamp, sequenceNumber, pkName, bucketNum, columns,
                 stats, schemaName, parentTableName, indexes, isImmutableRows, physicalNames, defaultFamilyName,
-                viewExpression, disableWAL, multiTenant, storeNulls, viewType, viewIndexId, indexType, baseColumnCount, rowKeyOrderOptimizable, isTransactional);
+                viewExpression, disableWAL, multiTenant, storeNulls, viewType, viewIndexId, indexType, baseColumnCount, rowKeyOrderOptimizable,
+                isTransactional, updateCacheFrequency);
     }
 
     @Override
+    public long getUpdateCacheFrequency() {
+        return updateCacheFrequency;
+    }
+    
+    @Override
     public boolean isMultiTenant() {
         return multiTenant;
     }
@@ -331,7 +344,7 @@ public class PTableImpl implements PTable {
             PName pkName, Integer bucketNum, List<PColumn> columns, PTableStats stats, PName parentSchemaName, PName parentTableName,
             List<PTable> indexes, boolean isImmutableRows, List<PName> physicalNames, PName defaultFamilyName, String viewExpression, boolean disableWAL,
             boolean multiTenant, boolean storeNulls, ViewType viewType, Short viewIndexId,
-            IndexType indexType , int baseColumnCount, boolean rowKeyOrderOptimizable, boolean isTransactional) throws SQLException {
+            IndexType indexType , int baseColumnCount, boolean rowKeyOrderOptimizable, boolean isTransactional, long updateCacheFrequency) 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 +
@@ -363,6 +376,7 @@ public class PTableImpl implements PTable {
         this.isTransactional = isTransactional;
         this.tableStats = stats;
         this.rowKeyOrderOptimizable = rowKeyOrderOptimizable;
+        this.updateCacheFrequency = updateCacheFrequency;
         List<PColumn> pkColumns;
         PColumn[] allColumns;
         
@@ -1060,12 +1074,18 @@ public class PTableImpl implements PTable {
       if (table.hasRowKeyOrderOptimizable()) {
           rowKeyOrderOptimizable = table.getRowKeyOrderOptimizable();
       }
+      long updateCacheFrequency = 0;
+      if (table.hasUpdateCacheFrequency()) {
+          updateCacheFrequency = table.getUpdateCacheFrequency();
+      }
+      
       try {
         PTableImpl result = new PTableImpl();
         result.init(tenantId, schemaName, tableName, tableType, indexState, timeStamp, sequenceNumber, pkName,
-          (bucketNum == NO_SALTING) ? null : bucketNum, columns, stats, schemaName,dataTableName, indexes,
-              isImmutableRows, physicalNames, defaultFamilyName, viewStatement, disableWAL,
-                multiTenant, storeNulls, viewType, viewIndexId, indexType, baseColumnCount, rowKeyOrderOptimizable, isTransactional);
+            (bucketNum == NO_SALTING) ? null : bucketNum, columns, stats, schemaName,dataTableName, indexes,
+            isImmutableRows, physicalNames, defaultFamilyName, viewStatement, disableWAL,
+            multiTenant, storeNulls, viewType, viewIndexId, indexType, baseColumnCount, rowKeyOrderOptimizable,
+            isTransactional, updateCacheFrequency);
         return result;
       } catch (SQLException e) {
         throw new RuntimeException(e); // Impossible
@@ -1157,6 +1177,7 @@ public class PTableImpl implements PTable {
       }
       builder.setBaseColumnCount(table.getBaseColumnCount());
       builder.setRowKeyOrderOptimizable(table.rowKeyOrderOptimizable());
+      builder.setUpdateCacheFrequency(table.getUpdateCacheFrequency());
 
       return builder.build();
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/59b336ec/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableRef.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableRef.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableRef.java
index 83d0b42..c4bc510 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableRef.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableRef.java
@@ -20,15 +20,16 @@ package org.apache.phoenix.schema;
 public class PTableRef {
     private final PTable table;
     private final int estSize;
+    private final long createTime;
+    private final long resolvedTimeStamp;
 	private volatile long lastAccessTime;
-	// timestamp (scn or txn timestamp at which rpc to fetch the table was made)
-    private long resolvedTimeStamp;
     
     public PTableRef(PTable table, long lastAccessTime, int estSize, long resolvedTime) {
         this.table = table;
         this.lastAccessTime = lastAccessTime;
         this.estSize = estSize;
         this.resolvedTimeStamp = resolvedTime;
+        this.createTime = lastAccessTime;
     }
 
     public PTableRef(PTable table, long lastAccessTime, long resolvedTime) {
@@ -39,6 +40,14 @@ public class PTableRef {
         this (tableRef.table, tableRef.lastAccessTime, tableRef.estSize, tableRef.resolvedTimeStamp);
     }
     
+    /**
+     * Tracks how long this entry has been in the cache
+     * @return time in milliseconds for how long this entry has been in the cache.
+     */
+    public long getCreateTime() {
+        return createTime;
+    }
+    
     public PTable getTable() {
 		return table;
 	}
@@ -58,8 +67,4 @@ public class PTableRef {
 	public void setLastAccessTime(long lastAccessTime) {
 		this.lastAccessTime = lastAccessTime;
 	}
-	
-	public void setResolvedTimeStamp(long resolvedTimeStamp) {
-		this.resolvedTimeStamp = resolvedTimeStamp;
-	}
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/59b336ec/phoenix-core/src/main/java/org/apache/phoenix/schema/TableProperty.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/TableProperty.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/TableProperty.java
index 091b929..5967d8b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/TableProperty.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/TableProperty.java
@@ -26,6 +26,7 @@ import static org.apache.phoenix.exception.SQLExceptionCode.VIEW_WITH_PROPERTIES
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DEFAULT_COLUMN_FAMILY_NAME;
 
 import java.sql.SQLException;
+import java.util.Map;
 
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.phoenix.exception.SQLExceptionCode;
@@ -49,6 +50,23 @@ public enum TableProperty {
     STORE_NULLS(PhoenixDatabaseMetaData.STORE_NULLS, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, true, false),
     
     TRANSACTIONAL(PhoenixDatabaseMetaData.TRANSACTIONAL, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, true, false),
+
+    UPDATE_CACHE_FREQUENCY(PhoenixDatabaseMetaData.UPDATE_CACHE_FREQUENCY, true, true) {
+	    @Override
+        public Object getValue(Object value) {
+	        if (value instanceof String) {
+	            String strValue = (String) value;
+	            if ("ALWAYS".equalsIgnoreCase(strValue)) {
+	                return 0L;
+	            } else if ("NEVER".equalsIgnoreCase(strValue)) {
+	                return Long.MAX_VALUE;
+	            }
+	        } else {
+	            return value == null ? null : ((Number) value).longValue();
+	        }
+	        return value;
+	    }	    
+	},
     ;
 
 
@@ -87,6 +105,14 @@ public enum TableProperty {
 		return true;
 	}
 
+	public Object getValue(Object value) {
+	    return value;
+	}
+	
+    public Object getValue(Map<String, Object> props) {
+        return getValue(props.get(this.toString()));
+    }
+    
 	// isQualified is true if column family name is specified in property name
 	public void validate(boolean isMutating, boolean isQualified, PTableType tableType) throws SQLException {
 		checkForColumnFamily(isQualified);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/59b336ec/phoenix-core/src/test/java/org/apache/phoenix/execute/CorrelatePlanTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/execute/CorrelatePlanTest.java b/phoenix-core/src/test/java/org/apache/phoenix/execute/CorrelatePlanTest.java
index 9d5e17c..72f3e01 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/execute/CorrelatePlanTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/execute/CorrelatePlanTest.java
@@ -36,12 +36,12 @@ import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.phoenix.compile.ColumnResolver;
 import org.apache.phoenix.compile.FromCompiler;
 import org.apache.phoenix.compile.JoinCompiler;
+import org.apache.phoenix.compile.OrderByCompiler.OrderBy;
 import org.apache.phoenix.compile.QueryPlan;
 import org.apache.phoenix.compile.RowProjector;
 import org.apache.phoenix.compile.SequenceManager;
 import org.apache.phoenix.compile.StatementContext;
 import org.apache.phoenix.compile.TupleProjectionCompiler;
-import org.apache.phoenix.compile.OrderByCompiler.OrderBy;
 import org.apache.phoenix.coprocessor.MetaDataProtocol;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.expression.ComparisonExpression;
@@ -233,7 +233,7 @@ public class CorrelatePlanTest {
                     PTableType.SUBQUERY, null, MetaDataProtocol.MIN_TABLE_TIMESTAMP, PTable.INITIAL_SEQ_NUM,
                     null, null, columns, null, null, Collections.<PTable>emptyList(),
                     false, Collections.<PName>emptyList(), null, null, false, false, false, null,
-                    null, null, true, false);
+                    null, null, true, false, 0);
             TableRef sourceTable = new TableRef(pTable);
             List<ColumnRef> sourceColumnRefs = Lists.<ColumnRef> newArrayList();
             for (PColumn column : sourceTable.getTable().getColumns()) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/59b336ec/phoenix-core/src/test/java/org/apache/phoenix/schema/PMetaDataImplTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/schema/PMetaDataImplTest.java b/phoenix-core/src/test/java/org/apache/phoenix/schema/PMetaDataImplTest.java
index 14b5f1f..9c92679 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/schema/PMetaDataImplTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/schema/PMetaDataImplTest.java
@@ -17,29 +17,37 @@
  */
 package org.apache.phoenix.schema;
 
-import com.google.common.collect.Sets;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.phoenix.util.TimeKeeper;
-import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
 
 import java.sql.SQLException;
 import java.util.Set;
 
-import static org.junit.Assert.assertEquals;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.phoenix.util.TimeKeeper;
+import org.junit.Test;
+
+import com.google.common.collect.Sets;
 
 public class PMetaDataImplTest {
     
-    private static PMetaData addToTable(PMetaData metaData, String name, int size) throws SQLException {
+    private static PMetaData addToTable(PMetaData metaData, String name, int size, TestTimeKeeper timeKeeper) throws SQLException {
         PTable table = new PSizedTable(new PTableKey(null,name), size);
-        return metaData.addTable(table, System.currentTimeMillis());
+        PMetaData newMetaData = metaData.addTable(table, System.currentTimeMillis());
+        timeKeeper.incrementTime();
+        return newMetaData;
     }
     
-    private static PMetaData removeFromTable(PMetaData metaData, String name) throws SQLException {
-        return metaData.removeTable(null, name, null, HConstants.LATEST_TIMESTAMP);
+    private static PMetaData removeFromTable(PMetaData metaData, String name, TestTimeKeeper timeKeeper) throws SQLException {
+        PMetaData newMetaData =  metaData.removeTable(null, name, null, HConstants.LATEST_TIMESTAMP);
+        timeKeeper.incrementTime();
+        return newMetaData;
     }
     
-    private static PTable getFromTable(PMetaData metaData, String name) throws TableNotFoundException {
-        return metaData.getTableRef(new PTableKey(null,name)).getTable();
+    private static PTable getFromTable(PMetaData metaData, String name, TestTimeKeeper timeKeeper) throws TableNotFoundException {
+        PTable table = metaData.getTableRef(new PTableKey(null,name)).getTable();
+        timeKeeper.incrementTime();
+        return table;
     }
     
     private static void assertNames(PMetaData metaData, String... names) {
@@ -56,50 +64,54 @@ public class PMetaDataImplTest {
         
         @Override
         public long getCurrentTime() {
-            return time++;
+            return time;
         }
         
+        public void incrementTime() {
+            time++;
+        }
     }
     
     @Test
     public void testEviction() throws Exception {
         long maxSize = 10;
-        PMetaData metaData = new PMetaDataImpl(5, maxSize, new TestTimeKeeper());
-        metaData = addToTable(metaData, "a", 5);
+        TestTimeKeeper timeKeeper = new TestTimeKeeper();
+        PMetaData metaData = new PMetaDataImpl(5, maxSize, timeKeeper);
+        metaData = addToTable(metaData, "a", 5, timeKeeper);
         assertEquals(1, metaData.size());
-        metaData = addToTable(metaData, "b", 4);
+        metaData = addToTable(metaData, "b", 4, timeKeeper);
         assertEquals(2, metaData.size());
-        metaData = addToTable(metaData, "c", 3);
+        metaData = addToTable(metaData, "c", 3, timeKeeper);
         assertEquals(2, metaData.size());
         assertNames(metaData, "b","c");
 
-        metaData = addToTable(metaData, "b", 8);
+        metaData = addToTable(metaData, "b", 8, timeKeeper);
         assertEquals(1, metaData.size());
         assertNames(metaData, "b");
 
-        metaData = addToTable(metaData, "d", 11);
+        metaData = addToTable(metaData, "d", 11, timeKeeper);
         assertEquals(1, metaData.size());
         assertNames(metaData, "d");
         
-        metaData = removeFromTable(metaData, "d");
+        metaData = removeFromTable(metaData, "d", timeKeeper);
         assertNames(metaData);
         
-        metaData = addToTable(metaData, "a", 4);
+        metaData = addToTable(metaData, "a", 4, timeKeeper);
         assertEquals(1, metaData.size());
-        metaData = addToTable(metaData, "b", 3);
+        metaData = addToTable(metaData, "b", 3, timeKeeper);
         assertEquals(2, metaData.size());
-        metaData = addToTable(metaData, "c", 2);
+        metaData = addToTable(metaData, "c", 2, timeKeeper);
         assertEquals(3, metaData.size());
         assertNames(metaData, "a", "b","c");
         
-        getFromTable(metaData, "a");
-        metaData = addToTable(metaData, "d", 3);
+        getFromTable(metaData, "a", timeKeeper);
+        metaData = addToTable(metaData, "d", 3, timeKeeper);
         assertEquals(3, metaData.size());
         assertNames(metaData, "c", "a","d");
         
         // Clone maintains insert order
         metaData = metaData.clone();
-        metaData = addToTable(metaData, "e", 6);
+        metaData = addToTable(metaData, "e", 6, timeKeeper);
         assertEquals(2, metaData.size());
         assertNames(metaData, "d","e");
     }
@@ -107,18 +119,19 @@ public class PMetaDataImplTest {
     @Test
     public void shouldNotEvictMoreEntriesThanNecessary() throws Exception {
         long maxSize = 5;
-        PMetaData metaData = new PMetaDataImpl(5, maxSize, new TestTimeKeeper());
-        metaData = addToTable(metaData, "a", 1);
+        TestTimeKeeper timeKeeper = new TestTimeKeeper();
+        PMetaData metaData = new PMetaDataImpl(5, maxSize, timeKeeper);
+        metaData = addToTable(metaData, "a", 1, timeKeeper);
         assertEquals(1, metaData.size());
-        metaData = addToTable(metaData, "b", 1);
+        metaData = addToTable(metaData, "b", 1, timeKeeper);
         assertEquals(2, metaData.size());
         assertNames(metaData, "a", "b");
-        metaData = addToTable(metaData, "c", 3);
+        metaData = addToTable(metaData, "c", 3, timeKeeper);
         assertEquals(3, metaData.size());
         assertNames(metaData, "a", "b", "c");
-        getFromTable(metaData, "a");
-        getFromTable(metaData, "b");
-        metaData = addToTable(metaData, "d", 3);
+        getFromTable(metaData, "a", timeKeeper);
+        getFromTable(metaData, "b", timeKeeper);
+        metaData = addToTable(metaData, "d", 3, timeKeeper);
         assertEquals(3, metaData.size());
         assertNames(metaData, "a", "b", "d");
     }
@@ -126,19 +139,20 @@ public class PMetaDataImplTest {
     @Test
     public void shouldAlwaysKeepAtLeastOneEntryEvenIfTooLarge() throws Exception {
         long maxSize = 5;
-        PMetaData metaData = new PMetaDataImpl(5, maxSize, new TestTimeKeeper());
-        metaData = addToTable(metaData, "a", 1);
+        TestTimeKeeper timeKeeper = new TestTimeKeeper();
+        PMetaData metaData = new PMetaDataImpl(5, maxSize, timeKeeper);
+        metaData = addToTable(metaData, "a", 1, timeKeeper);
         assertEquals(1, metaData.size());
-        metaData = addToTable(metaData, "b", 1);
+        metaData = addToTable(metaData, "b", 1, timeKeeper);
         assertEquals(2, metaData.size());
-        metaData = addToTable(metaData, "c", 5);
+        metaData = addToTable(metaData, "c", 5, timeKeeper);
         assertEquals(1, metaData.size());
-        metaData = addToTable(metaData, "d", 20);
+        metaData = addToTable(metaData, "d", 20, timeKeeper);
         assertEquals(1, metaData.size());
         assertNames(metaData, "d");
-        metaData = addToTable(metaData, "e", 1);
+        metaData = addToTable(metaData, "e", 1, timeKeeper);
         assertEquals(1, metaData.size());
-        metaData = addToTable(metaData, "f", 2);
+        metaData = addToTable(metaData, "f", 2, timeKeeper);
         assertEquals(2, metaData.size());
         assertNames(metaData, "e", "f");
     }
@@ -146,23 +160,42 @@ public class PMetaDataImplTest {
     @Test
     public void shouldAlwaysKeepOneEntryIfMaxSizeIsZero() throws Exception {
         long maxSize = 0;
-        PMetaData metaData = new PMetaDataImpl(0, maxSize, new TestTimeKeeper());
-        metaData = addToTable(metaData, "a", 1);
+        TestTimeKeeper timeKeeper = new TestTimeKeeper();
+        PMetaData metaData = new PMetaDataImpl(0, maxSize, timeKeeper);
+        metaData = addToTable(metaData, "a", 1, timeKeeper);
         assertEquals(1, metaData.size());
-        metaData = addToTable(metaData, "b", 1);
+        metaData = addToTable(metaData, "b", 1, timeKeeper);
         assertEquals(1, metaData.size());
-        metaData = addToTable(metaData, "c", 5);
+        metaData = addToTable(metaData, "c", 5, timeKeeper);
         assertEquals(1, metaData.size());
-        metaData = addToTable(metaData, "d", 20);
+        metaData = addToTable(metaData, "d", 20, timeKeeper);
         assertEquals(1, metaData.size());
         assertNames(metaData, "d");
-        metaData = addToTable(metaData, "e", 1);
+        metaData = addToTable(metaData, "e", 1, timeKeeper);
         assertEquals(1, metaData.size());
-        metaData = addToTable(metaData, "f", 2);
+        metaData = addToTable(metaData, "f", 2, timeKeeper);
         assertEquals(1, metaData.size());
         assertNames(metaData, "f");
     }
 
+    @Test
+    public void testAge() throws Exception {
+        long maxSize = 10;
+        TestTimeKeeper timeKeeper = new TestTimeKeeper();
+        PMetaData metaData = new PMetaDataImpl(5, maxSize, timeKeeper);
+        String tableName = "a";
+        metaData = addToTable(metaData, tableName, 1, timeKeeper);
+        PTableRef aTableRef = metaData.getTableRef(new PTableKey(null,tableName));
+        assertNotNull(aTableRef);
+        assertEquals(1, metaData.getAge(aTableRef));
+        tableName = "b";
+        metaData = addToTable(metaData, tableName, 1, timeKeeper);
+        PTableRef bTableRef = metaData.getTableRef(new PTableKey(null,tableName));
+        assertNotNull(bTableRef);
+        assertEquals(1, metaData.getAge(bTableRef));
+        assertEquals(2, metaData.getAge(aTableRef));
+    }
+    
     private static class PSizedTable extends PTableImpl {
         private final int size;
         private final PTableKey key;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/59b336ec/phoenix-protocol/src/main/PTable.proto
----------------------------------------------------------------------
diff --git a/phoenix-protocol/src/main/PTable.proto b/phoenix-protocol/src/main/PTable.proto
index 3048a40..485dd11 100644
--- a/phoenix-protocol/src/main/PTable.proto
+++ b/phoenix-protocol/src/main/PTable.proto
@@ -87,4 +87,5 @@ message PTable {
   optional int32 baseColumnCount = 25;
   optional bool rowKeyOrderOptimizable = 26;
   optional bool transactional = 27;
+  optional int64 updateCacheFrequency = 28;
 }


[06/50] [abbrv] phoenix git commit: PHOENIX-2478 Rows committed in transaction overlapping index creation are not populated

Posted by ma...@apache.org.
PHOENIX-2478 Rows committed in transaction overlapping index creation are not populated


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

Branch: refs/heads/calcite
Commit: 3520e12858223dfcad343e0a3a29c71fe4d074bc
Parents: 1369937
Author: James Taylor <jt...@salesforce.com>
Authored: Tue Jan 19 08:19:44 2016 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Tue Jan 19 08:19:44 2016 -0800

----------------------------------------------------------------------
 .../phoenix/end2end/index/ImmutableIndexIT.java | 239 ++++++++++++++-----
 .../apache/phoenix/execute/MutationState.java   | 115 +++++----
 .../java/org/apache/phoenix/query/BaseTest.java | 103 +++-----
 3 files changed, 285 insertions(+), 172 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/3520e128/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
index 0d329fe..c4ecfbb 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
@@ -20,7 +20,6 @@ package org.apache.phoenix.end2end.index;
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
 
 import java.sql.Connection;
 import java.sql.DriverManager;
@@ -29,12 +28,26 @@ import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.List;
 import java.util.Map;
 import java.util.Properties;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HBaseIOException;
+import org.apache.hadoop.hbase.client.Durability;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
 import org.apache.phoenix.end2end.Shadower;
-import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.query.BaseTest;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.util.PropertiesUtil;
@@ -47,80 +60,194 @@ import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
+import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
 
 @RunWith(Parameterized.class)
 public class ImmutableIndexIT extends BaseHBaseManagedTimeIT {
-	
-	private final boolean localIndex;
-	private final String tableDDLOptions;
-	private final String tableName;
+
+    private final boolean localIndex;
+    private final String tableDDLOptions;
+    private final String tableName;
     private final String indexName;
     private final String fullTableName;
     private final String fullIndexName;
-	
-	public ImmutableIndexIT(boolean localIndex, boolean transactional) {
-		this.localIndex = localIndex;
-		StringBuilder optionBuilder = new StringBuilder("IMMUTABLE_ROWS=true");
-		if (transactional) {
-			optionBuilder.append(", TRANSACTIONAL=true");
-		}
-		this.tableDDLOptions = optionBuilder.toString();
-		this.tableName = TestUtil.DEFAULT_DATA_TABLE_NAME + ( transactional ?  "_TXN" : "");
+
+    private static String TABLE_NAME;
+    private static String INDEX_DDL;
+    public static final AtomicInteger NUM_ROWS = new AtomicInteger(1);
+
+    public ImmutableIndexIT(boolean localIndex, boolean transactional) {
+        this.localIndex = localIndex;
+        StringBuilder optionBuilder = new StringBuilder("IMMUTABLE_ROWS=true");
+        if (transactional) {
+            optionBuilder.append(", TRANSACTIONAL=true");
+        }
+        this.tableDDLOptions = optionBuilder.toString();
+        this.tableName = TestUtil.DEFAULT_DATA_TABLE_NAME + ( transactional ?  "_TXN" : "");
         this.indexName = "IDX" + ( transactional ?  "_TXN" : "");
         this.fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
         this.fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
-	}
-	
-	@BeforeClass
+    }
+
+    @BeforeClass
     @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
     public static void doSetup() throws Exception {
-        Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
-        props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
-        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+        Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(1);
+        serverProps.put("hbase.coprocessor.region.classes", CreateIndexRegionObserver.class.getName());
+        Map<String, String> clientProps = Maps.newHashMapWithExpectedSize(2);
+        clientProps.put(QueryServices.TRANSACTIONS_ENABLED, "true");
+        setUpTestDriver(new ReadOnlyProps(serverProps.entrySet().iterator()), new ReadOnlyProps(clientProps.entrySet().iterator()));
     }
-	
-	@Parameters(name="localIndex = {0} , transactional = {1}")
+
+    @Parameters(name="localIndex = {0} , transactional = {1}")
     public static Collection<Boolean[]> data() {
         return Arrays.asList(new Boolean[][] {     
-                 { false, false }, { false, true }, { true, false }, { true, true }
-           });
+            { false, true }, { true, true }
+        });
     }
-   
+
+
     @Test
-    public void testDropIfImmutableKeyValueColumn() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+    public void testCreateIndexDuringUpsertSelect() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        props.setProperty(QueryServices.MUTATE_BATCH_SIZE_ATTRIB, Integer.toString(100));
+        TABLE_NAME = fullTableName + "_testCreateIndexDuringUpsertSelect";
+        String ddl ="CREATE TABLE " + TABLE_NAME + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+        INDEX_DDL = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX IF NOT EXISTS " + indexName + " ON " + TABLE_NAME
+                + " (long_pk, varchar_pk)"
+                + " INCLUDE (long_col1, long_col2)";
+
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            conn.setAutoCommit(false);
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+
+            upsertRows(conn, TABLE_NAME, 220);
+            conn.commit();
+
+            // run the upsert select and also create an index
+            conn.setAutoCommit(true);
+            String upsertSelect = "UPSERT INTO " + TABLE_NAME + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk) " + 
+                    "SELECT varchar_pk||'_upsert_select', char_pk, int_pk, long_pk, decimal_pk, date_pk FROM "+ TABLE_NAME;    
+            conn.createStatement().execute(upsertSelect);
+
+            ResultSet rs;
+            rs = conn.createStatement().executeQuery("SELECT /*+ NO_INDEX */ COUNT(*) FROM " + TABLE_NAME);
+            assertTrue(rs.next());
+            assertEquals(440,rs.getInt(1));
+            rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + TABLE_NAME);
+            assertTrue(rs.next());
+            assertEquals(440,rs.getInt(1));
+        }
+        finally {
+            conn.close();
+        }
+    }
+
+    // used to create an index while a batch of rows are being written
+    public static class CreateIndexRegionObserver extends SimpleRegionObserver {
+        @Override
+        public void postPut(ObserverContext<RegionCoprocessorEnvironment> c,
+                Put put, WALEdit edit, final Durability durability)
+                        throws HBaseIOException {
+            String tableName = c.getEnvironment().getRegion().getRegionInfo()
+                    .getTable().getNameAsString();
+            if (tableName.equalsIgnoreCase(TABLE_NAME)
+                    // create the index after the second batch of 1000 rows
+                    && Bytes.startsWith(put.getRow(), Bytes.toBytes("varchar200_upsert_select"))) {
+                try {
+                    Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+                    try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+                        conn.createStatement().execute(INDEX_DDL);
+                    }
+                } catch (SQLException e) {
+                    throw new DoNotRetryIOException(e);
+                } 
+            }
+        }
+    }
+
+    private static class UpsertRunnable implements Runnable {
+        private static final int NUM_ROWS_IN_BATCH = 10000;
+        private final String fullTableName;
+
+        public UpsertRunnable(String fullTableName) {
+            this.fullTableName = fullTableName;
+        }
+
+        public void run() {
+            Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+            try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+                while (true) {
+                    // write a large batch of rows
+                    boolean fistRowInBatch = true;
+                    for (int i=0; i<NUM_ROWS_IN_BATCH; ++i) {
+                        BaseTest.upsertRow(conn, fullTableName, NUM_ROWS.intValue(), fistRowInBatch);
+                        NUM_ROWS.incrementAndGet();
+                        fistRowInBatch = false;
+                    }
+                    conn.commit();
+                    Thread.sleep(500);
+                }
+            } catch (SQLException e) {
+                throw new RuntimeException(e);
+            } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+            }
+        }
+    }
+
+    @Test
+    public void testCreateIndexWhileUpsertingData() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+        String indexDDL = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX IF NOT EXISTS " + indexName + " ON " + fullTableName
+                + " (long_pk, varchar_pk)"
+                + " INCLUDE (long_col1, long_col2)";
+        int numThreads = 3;
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (long_col1)";
-	        stmt.execute(ddl);
-	        
-	        ResultSet rs;
-	        
-	        rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullTableName);
-	        assertTrue(rs.next());
-	        assertEquals(3,rs.getInt(1));
-	        rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullIndexName);
-	        assertTrue(rs.next());
-	        assertEquals(3,rs.getInt(1));
-	        
-	        conn.setAutoCommit(true);
-	        String dml = "DELETE from " + fullTableName + " WHERE long_col2 = 4";
-	        try {
-	            conn.createStatement().execute(dml);
-	            fail();
-	        } catch (SQLException e) {
-	            assertEquals(SQLExceptionCode.INVALID_FILTER_ON_IMMUTABLE_ROWS.getErrorCode(), e.getErrorCode());
-	        }
-	            
-	        conn.createStatement().execute("DROP TABLE " + fullTableName);
+            conn.setAutoCommit(false);
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+
+            ExecutorService threadPool = Executors.newFixedThreadPool(numThreads);
+            List<Future<?>> futureList = Lists.newArrayListWithExpectedSize(numThreads);
+            for (int i =0; i<numThreads; ++i) {
+                futureList.add(threadPool.submit(new UpsertRunnable(fullTableName)));
+            }
+            // upsert some rows before creating the index 
+            Thread.sleep(5000);
+
+            // create the index 
+            try (Connection conn2 = DriverManager.getConnection(getUrl(), props)) {
+                conn2.setAutoCommit(false);
+                Statement stmt2 = conn2.createStatement();
+                stmt2.execute(indexDDL);
+                conn2.commit();
+            }
+
+            // upsert some rows after creating the index
+            Thread.sleep(1000);
+            // cancel the running threads
+            for (Future<?> future : futureList) {
+                future.cancel(true);
+            }
+            threadPool.shutdownNow();
+            threadPool.awaitTermination(30, TimeUnit.SECONDS);
+            Thread.sleep(1000);
+
+            ResultSet rs;
+            rs = conn.createStatement().executeQuery("SELECT /*+ NO_INDEX */ COUNT(*) FROM " + fullTableName);
+            assertTrue(rs.next());
+            int dataTableRowCount = rs.getInt(1);
+            rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullIndexName);
+            assertTrue(rs.next());
+            int indexTableRowCount = rs.getInt(1);
+            assertEquals("Data and Index table should have the same number of rows ", dataTableRowCount, indexTableRowCount);
         }
     }
-    
 
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3520e128/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
index a6fe98d..8caac5d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
@@ -135,7 +135,7 @@ public class MutationState implements SQLCloseable {
     private int numRows = 0;
     private int[] uncommittedStatementIndexes = EMPTY_STATEMENT_INDEX_ARRAY;
     private boolean isExternalTxContext = false;
-    private Map<TableRef, Map<ImmutableBytesPtr,RowMutationState>> txMutations;
+    private Map<TableRef, Map<ImmutableBytesPtr,RowMutationState>> txMutations = Collections.emptyMap();
     
     private final MutationMetricQueue mutationMetricQueue;
     private ReadMetricQueue readMetricQueue;
@@ -435,6 +435,59 @@ public class MutationState implements SQLCloseable {
         return sizeOffset + numRows;
     }
     
+    private void joinMutationState(TableRef tableRef, Map<ImmutableBytesPtr,RowMutationState> srcRows,
+            Map<TableRef, Map<ImmutableBytesPtr, RowMutationState>> dstMutations) {
+        PTable table = tableRef.getTable();
+        boolean isIndex = table.getType() == PTableType.INDEX;
+        boolean incrementRowCount = dstMutations == this.mutations;
+        Map<ImmutableBytesPtr,RowMutationState> existingRows = dstMutations.put(tableRef, srcRows);
+        if (existingRows != null) { // Rows for that table already exist
+            // Loop through new rows and replace existing with new
+            for (Map.Entry<ImmutableBytesPtr,RowMutationState> rowEntry : srcRows.entrySet()) {
+                // Replace existing row with new row
+                RowMutationState existingRowMutationState = existingRows.put(rowEntry.getKey(), rowEntry.getValue());
+                if (existingRowMutationState != null) {
+                    Map<PColumn,byte[]> existingValues = existingRowMutationState.getColumnValues();
+                    if (existingValues != PRow.DELETE_MARKER) {
+                        Map<PColumn,byte[]> newRow = rowEntry.getValue().getColumnValues();
+                        // if new row is PRow.DELETE_MARKER, it means delete, and we don't need to merge it with existing row. 
+                        if (newRow != PRow.DELETE_MARKER) {
+                            // Merge existing column values with new column values
+                            existingRowMutationState.join(rowEntry.getValue());
+                            // Now that the existing row has been merged with the new row, replace it back
+                            // again (since it was merged with the new one above).
+                            existingRows.put(rowEntry.getKey(), existingRowMutationState);
+                        }
+                    }
+                } else {
+                    if (incrementRowCount && !isIndex) { // Don't count index rows in row count
+                        numRows++;
+                    }
+                }
+            }
+            // Put the existing one back now that it's merged
+            dstMutations.put(tableRef, existingRows);
+        } else {
+            // Size new map at batch size as that's what it'll likely grow to.
+            Map<ImmutableBytesPtr,RowMutationState> newRows = Maps.newHashMapWithExpectedSize(connection.getMutateBatchSize());
+            newRows.putAll(srcRows);
+            dstMutations.put(tableRef, newRows);
+            if (incrementRowCount && !isIndex) {
+                numRows += srcRows.size();
+            }
+        }
+    }
+    
+    private void joinMutationState(Map<TableRef, Map<ImmutableBytesPtr, RowMutationState>> srcMutations, 
+            Map<TableRef, Map<ImmutableBytesPtr, RowMutationState>> dstMutations) {
+        // Merge newMutation with this one, keeping state from newMutation for any overlaps
+        for (Map.Entry<TableRef, Map<ImmutableBytesPtr,RowMutationState>> entry : srcMutations.entrySet()) {
+            // Replace existing entries for the table with new entries
+            TableRef tableRef = entry.getKey();
+            Map<ImmutableBytesPtr,RowMutationState> srcRows = entry.getValue();
+            joinMutationState(tableRef, srcRows, dstMutations);
+        }
+    }
     /**
      * Combine a newer mutation with this one, where in the event of overlaps, the newer one will take precedence.
      * Combine any metrics collected for the newer mutation.
@@ -453,48 +506,12 @@ public class MutationState implements SQLCloseable {
             txAwares.addAll(newMutationState.txAwares);
         }
         this.sizeOffset += newMutationState.sizeOffset;
-        // Merge newMutation with this one, keeping state from newMutation for any overlaps
-        for (Map.Entry<TableRef, Map<ImmutableBytesPtr,RowMutationState>> entry : newMutationState.mutations.entrySet()) {
-            // Replace existing entries for the table with new entries
-            TableRef tableRef = entry.getKey();
-            PTable table = tableRef.getTable();
-            boolean isIndex = table.getType() == PTableType.INDEX;
-            Map<ImmutableBytesPtr,RowMutationState> existingRows = this.mutations.put(tableRef, entry.getValue());
-            if (existingRows != null) { // Rows for that table already exist
-                // Loop through new rows and replace existing with new
-                for (Map.Entry<ImmutableBytesPtr,RowMutationState> rowEntry : entry.getValue().entrySet()) {
-                    // Replace existing row with new row
-                    RowMutationState existingRowMutationState = existingRows.put(rowEntry.getKey(), rowEntry.getValue());
-                    if (existingRowMutationState != null) {
-                        Map<PColumn,byte[]> existingValues = existingRowMutationState.getColumnValues();
-                        if (existingValues != PRow.DELETE_MARKER) {
-                            Map<PColumn,byte[]> newRow = rowEntry.getValue().getColumnValues();
-                            // if new row is PRow.DELETE_MARKER, it means delete, and we don't need to merge it with existing row. 
-                            if (newRow != PRow.DELETE_MARKER) {
-                                // Merge existing column values with new column values
-                                existingRowMutationState.join(rowEntry.getValue());
-                                // Now that the existing row has been merged with the new row, replace it back
-                                // again (since it was merged with the new one above).
-                                existingRows.put(rowEntry.getKey(), existingRowMutationState);
-                            }
-                        }
-                    } else {
-                        if (!isIndex) { // Don't count index rows in row count
-                            numRows++;
-                        }
-                    }
-                }
-                // Put the existing one back now that it's merged
-                this.mutations.put(entry.getKey(), existingRows);
-            } else {
-                // Size new map at batch size as that's what it'll likely grow to.
-                Map<ImmutableBytesPtr,RowMutationState> newRows = Maps.newHashMapWithExpectedSize(connection.getMutateBatchSize());
-                newRows.putAll(entry.getValue());
-                this.mutations.put(tableRef, newRows);
-                if (!isIndex) {
-                    numRows += entry.getValue().size();
-                }
+        joinMutationState(newMutationState.mutations, this.mutations);
+        if (!newMutationState.txMutations.isEmpty()) {
+            if (txMutations.isEmpty()) {
+                txMutations = Maps.newHashMapWithExpectedSize(mutations.size());
             }
+            joinMutationState(newMutationState.txMutations, this.txMutations);
         }
         mutationMetricQueue.combineMetricQueues(newMutationState.mutationMetricQueue);
         if (readMetricQueue == null) {
@@ -915,6 +932,7 @@ public class MutationState implements SQLCloseable {
                             long startTime = System.currentTimeMillis();
                             child.addTimelineAnnotation("Attempt " + retryCount);
                             hTable.batch(mutationList);
+                            if (logger.isDebugEnabled()) logger.debug("Sent batch of " + numMutations + " for " + Bytes.toString(htableName));
                             child.stop();
                             child.stop();
                             shouldRetry = false;
@@ -980,13 +998,13 @@ public class MutationState implements SQLCloseable {
                 // committed in the event of a failure.
                 if (isTransactional) {
                     addUncommittedStatementIndexes(valuesMap.values());
-                    if (txMutations == null) {
+                    if (txMutations.isEmpty()) {
                         txMutations = Maps.newHashMapWithExpectedSize(mutations.size());
                     }
                     // Keep all mutations we've encountered until a commit or rollback.
                     // This is not ideal, but there's not good way to get the values back
                     // in the event that we need to replay the commit.
-                    txMutations.put(tableRef, valuesMap);
+                    joinMutationState(tableRef, valuesMap, txMutations);
                 }
                 // Remove batches as we process them
                 if (sendAll) {
@@ -1082,7 +1100,7 @@ public class MutationState implements SQLCloseable {
     private void resetTransactionalState() {
         tx = null;
         txAwares.clear();
-        txMutations = null;
+        txMutations = Collections.emptyMap();
         uncommittedPhysicalNames.clear();
         uncommittedStatementIndexes = EMPTY_STATEMENT_INDEX_ARRAY;
     }
@@ -1187,9 +1205,7 @@ public class MutationState implements SQLCloseable {
                 break;
             }
             retryCount++;
-            if (txMutations != null) {
-                mutations.putAll(txMutations);
-            }
+            mutations.putAll(txMutations);
         } while (true);
     }
 
@@ -1214,6 +1230,7 @@ public class MutationState implements SQLCloseable {
             if (result.getTable() == null) {
                 throw new TableNotFoundException(dataTable.getSchemaName().getString(), dataTable.getTableName().getString());
             }
+            tableRef.setTable(result.getTable());
             if (!result.wasUpdated()) {
                 if (logger.isInfoEnabled()) logger.info("No updates to " + dataTable.getName().getString() + " as of "  + timestamp);
                 continue;
@@ -1223,7 +1240,7 @@ public class MutationState implements SQLCloseable {
                 // that an index was dropped and recreated with the same name but different
                 // indexed/covered columns.
                 addedIndexes = (!oldIndexes.equals(result.getTable().getIndexes()));
-                if (logger.isInfoEnabled()) logger.info((addedIndexes ? "Updates " : "No updates ") + "as of "  + timestamp + " to " + dataTable.getName().getString() + " with indexes " + dataTable.getIndexes());
+                if (logger.isInfoEnabled()) logger.info((addedIndexes ? "Updates " : "No updates ") + "as of "  + timestamp + " to " + dataTable.getName().getString() + " with indexes " + tableRef.getTable().getIndexes());
             }
         }
         if (logger.isInfoEnabled()) logger.info((addedIndexes ? "Updates " : "No updates ") + "to indexes as of "  + getInitialWritePointer());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3520e128/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
index 35bb8ce..951bfce 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
@@ -17,6 +17,7 @@
  */
 package org.apache.phoenix.query;
 
+import static org.apache.phoenix.query.BaseTest.setUpConfigForMiniCluster;
 import static org.apache.phoenix.util.PhoenixRuntime.CURRENT_SCN_ATTRIB;
 import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL;
 import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL_TERMINATOR;
@@ -150,6 +151,7 @@ import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.hbase.index.balancer.IndexLoadBalancer;
 import org.apache.phoenix.hbase.index.master.IndexMasterObserver;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.jdbc.PhoenixDriver;
 import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver;
 import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver.ConnectionInfo;
 import org.apache.phoenix.jdbc.PhoenixTestDriver;
@@ -200,7 +202,7 @@ import com.google.inject.util.Providers;
 public abstract class BaseTest {
     protected static final String TEST_TABLE_SCHEMA = "(" +
             "   varchar_pk VARCHAR NOT NULL, " +
-            "   char_pk CHAR(6) NOT NULL, " +
+            "   char_pk CHAR(10) NOT NULL, " +
             "   int_pk INTEGER NOT NULL, "+ 
             "   long_pk BIGINT NOT NULL, " +
             "   decimal_pk DECIMAL(31, 10) NOT NULL, " +
@@ -1805,77 +1807,44 @@ public abstract class BaseTest {
     public HBaseTestingUtility getUtility() {
         return utility;
     }
+    
+    public static void upsertRows(Connection conn, String fullTableName, int numRows) throws SQLException {
+    	for (int i=1; i<=numRows; ++i) {
+	        upsertRow(conn, fullTableName, i, false);
+    	}
+    }
+
+    public static void upsertRow(Connection conn, String fullTableName, int index, boolean firstRowInBatch) throws SQLException {
+    	String upsert = "UPSERT INTO " + fullTableName
+                + " VALUES(?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
+		PreparedStatement stmt = conn.prepareStatement(upsert);
+		stmt.setString(1, firstRowInBatch ? "firstRowInBatch_" : "" + "varchar"+index);
+		stmt.setString(2, "char"+index);
+		stmt.setInt(3, index);
+		stmt.setLong(4, index);
+		stmt.setBigDecimal(5, new BigDecimal(index));
+		Date date = DateUtil.parseDate("2015-01-01 00:00:00");
+		stmt.setDate(6, date);
+		stmt.setString(7, "varchar_a");
+		stmt.setString(8, "chara");
+		stmt.setInt(9, index+1);
+		stmt.setLong(10, index+1);
+		stmt.setBigDecimal(11, new BigDecimal(index+1));
+		stmt.setDate(12, date);
+		stmt.setString(13, "varchar_b");
+		stmt.setString(14, "charb");
+		stmt.setInt(15, index+2);
+		stmt.setLong(16, index+2);
+		stmt.setBigDecimal(17, new BigDecimal(index+2));
+		stmt.setDate(18, date);
+		stmt.executeUpdate();
+	}
 
     // Populate the test table with data.
     public static void populateTestTable(String fullTableName) throws SQLException {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-            String upsert = "UPSERT INTO " + fullTableName
-                    + " VALUES(?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
-            PreparedStatement stmt = conn.prepareStatement(upsert);
-            stmt.setString(1, "varchar1");
-            stmt.setString(2, "char1");
-            stmt.setInt(3, 1);
-            stmt.setLong(4, 1L);
-            stmt.setBigDecimal(5, new BigDecimal(1.0));
-            Date date = DateUtil.parseDate("2015-01-01 00:00:00");
-            stmt.setDate(6, date);
-            stmt.setString(7, "varchar_a");
-            stmt.setString(8, "chara");
-            stmt.setInt(9, 2);
-            stmt.setLong(10, 2L);
-            stmt.setBigDecimal(11, new BigDecimal(2.0));
-            stmt.setDate(12, date);
-            stmt.setString(13, "varchar_b");
-            stmt.setString(14, "charb");
-            stmt.setInt(15, 3);
-            stmt.setLong(16, 3L);
-            stmt.setBigDecimal(17, new BigDecimal(3.0));
-            stmt.setDate(18, date);
-            stmt.executeUpdate();
-            
-            stmt.setString(1, "varchar2");
-            stmt.setString(2, "char2");
-            stmt.setInt(3, 2);
-            stmt.setLong(4, 2L);
-            stmt.setBigDecimal(5, new BigDecimal(2.0));
-            date = DateUtil.parseDate("2015-01-02 00:00:00");
-            stmt.setDate(6, date);
-            stmt.setString(7, "varchar_a");
-            stmt.setString(8, "chara");
-            stmt.setInt(9, 3);
-            stmt.setLong(10, 3L);
-            stmt.setBigDecimal(11, new BigDecimal(3.0));
-            stmt.setDate(12, date);
-            stmt.setString(13, "varchar_b");
-            stmt.setString(14, "charb");
-            stmt.setInt(15, 4);
-            stmt.setLong(16, 4L);
-            stmt.setBigDecimal(17, new BigDecimal(4.0));
-            stmt.setDate(18, date);
-            stmt.executeUpdate();
-            
-            stmt.setString(1, "varchar3");
-            stmt.setString(2, "char3");
-            stmt.setInt(3, 3);
-            stmt.setLong(4, 3L);
-            stmt.setBigDecimal(5, new BigDecimal(3.0));
-            date = DateUtil.parseDate("2015-01-03 00:00:00");
-            stmt.setDate(6, date);
-            stmt.setString(7, "varchar_a");
-            stmt.setString(8, "chara");
-            stmt.setInt(9, 4);
-            stmt.setLong(10, 4L);
-            stmt.setBigDecimal(11, new BigDecimal(4.0));
-            stmt.setDate(12, date);
-            stmt.setString(13, "varchar_b");
-            stmt.setString(14, "charb");
-            stmt.setInt(15, 5);
-            stmt.setLong(16, 5L);
-            stmt.setBigDecimal(17, new BigDecimal(5.0));
-            stmt.setDate(18, date);
-            stmt.executeUpdate();
-            
+        	upsertRows(conn, fullTableName, 3);
             conn.commit();
         }
     }


[48/50] [abbrv] phoenix git commit: Sync with master; Bug fix for ResultSet object of Float type; Walkaround for PHOENIX-2647

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/74409be8/phoenix-core/src/it/java/org/apache/phoenix/calcite/CalciteIndexIT.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/it/java/org/apache/phoenix/calcite/CalciteIndexIT.java
index 1197193,0000000..f200a24
mode 100644,000000..100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/calcite/CalciteIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/calcite/CalciteIndexIT.java
@@@ -1,283 -1,0 +1,283 @@@
 +/*
 + * 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.phoenix.calcite;
 +
 +import static org.junit.Assert.fail;
 +
 +import java.sql.Connection;
 +import java.sql.DriverManager;
 +import java.sql.SQLException;
 +import java.util.Arrays;
 +import java.util.Collection;
 +import java.util.Properties;
 +
 +import org.junit.Before;
 +import org.junit.Test;
 +import org.junit.runner.RunWith;
 +import org.junit.runners.Parameterized;
 +import org.junit.runners.Parameterized.Parameters;
 +
 +@RunWith(Parameterized.class)
 +public class CalciteIndexIT extends BaseCalciteIT {
 +    
 +    private final boolean localIndex;
 +    
 +    public CalciteIndexIT(boolean localIndex) {
 +        this.localIndex = localIndex;
 +    }
 +    
 +    @Parameters(name="localIndex = {0}")
 +    public static Collection<Boolean[]> data() {
 +        return Arrays.asList(new Boolean[][] {     
 +                 { false }, { true }
 +           });
 +    }
 +    
 +    @Before
 +    public void initTable() throws Exception {
 +        final String url = getUrl();
 +        final String index = localIndex ? "LOCAL INDEX" : "INDEX";
 +        initATableValues(getOrganizationId(), null, url);
 +        initSaltedTables(index);
 +        initMultiTenantTables(index);
 +        final Connection connection = DriverManager.getConnection(url);
 +        connection.createStatement().execute("CREATE " + index + " IF NOT EXISTS IDX1 ON aTable (a_string) INCLUDE (b_string, x_integer)");
 +        connection.createStatement().execute("CREATE " + index + " IF NOT EXISTS IDX2 ON aTable (b_string) INCLUDE (a_string, y_integer)");
 +        connection.createStatement().execute("CREATE " + index + " IF NOT EXISTS IDX_FULL ON aTable (b_string) INCLUDE (a_string, a_integer, a_date, a_time, a_timestamp, x_decimal, x_long, x_integer, y_integer, a_byte, a_short, a_float, a_double, a_unsigned_float, a_unsigned_double)");
 +        connection.createStatement().execute("UPDATE STATISTICS ATABLE");
 +        connection.createStatement().execute("UPDATE STATISTICS " + SALTED_TABLE_NAME);
 +        connection.createStatement().execute("UPDATE STATISTICS IDX_" + SALTED_TABLE_NAME);
 +        connection.createStatement().execute("UPDATE STATISTICS IDX1");
 +        connection.createStatement().execute("UPDATE STATISTICS IDX2");
 +        connection.createStatement().execute("UPDATE STATISTICS IDX_FULL");
 +        connection.close();
 +    }
 +    
 +    @Test public void testIndex() throws Exception {
 +        start(true).sql("select * from aTable where b_string = 'b'")
 +            .explainIs("PhoenixToEnumerableConverter\n" +
 +                       "  PhoenixServerProject(ORGANIZATION_ID=[$1], ENTITY_ID=[$2], A_STRING=[$3], B_STRING=[$0], A_INTEGER=[$4], A_DATE=[$5], A_TIME=[$6], A_TIMESTAMP=[$7], X_DECIMAL=[$8], X_LONG=[$9], X_INTEGER=[$10], Y_INTEGER=[$11], A_BYTE=[$12], A_SHORT=[$13], A_FLOAT=[$14], A_DOUBLE=[$15], A_UNSIGNED_FLOAT=[$16], A_UNSIGNED_DOUBLE=[$17])\n" +
 +                       "    PhoenixTableScan(table=[[phoenix, IDX_FULL]], filter=[=($0, 'b')])\n")
 +            .close();
 +        start(true).sql("select x_integer from aTable")
 +            .explainIs("PhoenixToEnumerableConverter\n" +
 +                       "  PhoenixServerProject(X_INTEGER=[$4])\n" +
 +                       "    PhoenixTableScan(table=[[phoenix, IDX1]])\n")
 +            .close();
 +        start(true).sql("select a_string from aTable order by a_string")
 +            .explainIs("PhoenixToEnumerableConverter\n" +
 +                       "  PhoenixServerProject(A_STRING=[$0])\n" +
 +                       "    PhoenixTableScan(table=[[phoenix, IDX1]], scanOrder=[FORWARD])\n")
 +            .close();
 +        start(true).sql("select a_string from aTable order by organization_id")
 +            .explainIs("PhoenixToEnumerableConverter\n" +
 +                       "  PhoenixServerProject(A_STRING=[$2], ORGANIZATION_ID=[$0])\n" +
 +                       "    PhoenixTableScan(table=[[phoenix, ATABLE]], scanOrder=[FORWARD])\n")
 +            .close();
 +        start(true).sql("select a_integer from aTable order by a_string")
 +            .explainIs("PhoenixToEnumerableConverter\n" +
 +                       "  PhoenixServerSort(sort0=[$1], dir0=[ASC])\n" +
 +                       "    PhoenixServerProject(A_INTEGER=[$4], A_STRING=[$2])\n" +
 +                       "      PhoenixTableScan(table=[[phoenix, ATABLE]])\n")
 +            .close();
 +        start(true).sql("select a_string, b_string from aTable where a_string = 'a'")
 +            .explainIs("PhoenixToEnumerableConverter\n" +
 +                       "  PhoenixServerProject(A_STRING=[$0], B_STRING=[$3])\n" +
 +                       "    PhoenixTableScan(table=[[phoenix, IDX1]], filter=[=($0, 'a')])\n")
 +            .close();
 +        start(true).sql("select a_string, b_string from aTable where b_string = 'b'")
 +            .explainIs("PhoenixToEnumerableConverter\n" +
 +                       "  PhoenixServerProject(A_STRING=[$3], B_STRING=[$0])\n" +
 +                       "    PhoenixTableScan(table=[[phoenix, IDX2]], filter=[=($0, 'b')])\n")
 +            .close();
 +        start(true).sql("select a_string, b_string, x_integer, y_integer from aTable where b_string = 'b'")
 +            .explainIs("PhoenixToEnumerableConverter\n" +
 +                       "  PhoenixServerProject(A_STRING=[$3], B_STRING=[$0], X_INTEGER=[$10], Y_INTEGER=[$11])\n" +
 +                       "    PhoenixTableScan(table=[[phoenix, IDX_FULL]], filter=[=($0, 'b')])\n")
 +            .close();
 +        start(true).sql("select a_string, count(*) from aTable group by a_string")
 +            .explainIs("PhoenixToEnumerableConverter\n" +
 +                       "  PhoenixServerAggregate(group=[{0}], EXPR$1=[COUNT()], isOrdered=[true])\n" +
 +                       "    PhoenixTableScan(table=[[phoenix, IDX1]], scanOrder=[FORWARD])\n")
 +            .close();
 +    }
 +    
 +    @Test public void testSaltedIndex() throws Exception {
 +        start(true).sql("select count(*) from " + NOSALT_TABLE_NAME + " where col0 > 3")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixServerAggregate(group=[{}], EXPR$0=[COUNT()])\n" +
 +                           "    PhoenixTableScan(table=[[phoenix, IDXSALTED_NOSALT_TEST_TABLE]], filter=[>(CAST($0):INTEGER, 3)])\n")
-                 .resultIs(new Object[][]{{2L}})
++                .resultIs(false, new Object[][]{{2L}})
 +                .close();
 +        start(true).sql("select mypk0, mypk1, col0 from " + NOSALT_TABLE_NAME + " where col0 <= 4")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixServerProject(MYPK0=[$1], MYPK1=[$2], COL0=[CAST($0):INTEGER])\n" +
 +                           "    PhoenixTableScan(table=[[phoenix, IDXSALTED_NOSALT_TEST_TABLE]], filter=[<=(CAST($0):INTEGER, 4)])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                        {2, 3, 4},
 +                        {1, 2, 3}})
 +                .close();
 +        start(true).sql("select * from " + SALTED_TABLE_NAME + " where mypk0 < 3")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixTableScan(table=[[phoenix, SALTED_TEST_TABLE]], filter=[<($0, 3)])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                        {1, 2, 3, 4},
 +                        {2, 3, 4, 5}})
 +                .close();
 +        start(true).sql("select count(*) from " + SALTED_TABLE_NAME + " where col0 > 3")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixServerAggregate(group=[{}], EXPR$0=[COUNT()])\n" +
 +                           "    PhoenixTableScan(table=[[phoenix, IDX_SALTED_TEST_TABLE]], filter=[>(CAST($0):INTEGER, 3)])\n")
-                 .resultIs(new Object[][]{{2L}})
++                .resultIs(false, new Object[][]{{2L}})
 +                .close();
 +        start(true).sql("select mypk0, mypk1, col0 from " + SALTED_TABLE_NAME + " where col0 <= 4")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixServerProject(MYPK0=[$1], MYPK1=[$2], COL0=[CAST($0):INTEGER])\n" +
 +                           "    PhoenixTableScan(table=[[phoenix, IDX_SALTED_TEST_TABLE]], filter=[<=(CAST($0):INTEGER, 4)])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                        {2, 3, 4},
 +                        {1, 2, 3}})
 +                .close();
 +        start(true).sql("select count(*) from " + SALTED_TABLE_NAME + " where col1 > 4")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixServerAggregate(group=[{}], EXPR$0=[COUNT()])\n" +
 +                           "    PhoenixTableScan(table=[[phoenix, IDXSALTED_SALTED_TEST_TABLE]], filter=[>(CAST($0):INTEGER, 4)])\n")
-                 .resultIs(new Object[][]{{2L}})
++                .resultIs(false, new Object[][]{{2L}})
 +                .close();
 +        start(true).sql("select * from " + SALTED_TABLE_NAME + " where col1 <= 5 order by col1")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixServerProject(MYPK0=[$1], MYPK1=[$2], COL0=[$3], COL1=[CAST($0):INTEGER])\n" +
 +                           "    PhoenixTableScan(table=[[phoenix, IDXSALTED_SALTED_TEST_TABLE]], filter=[<=(CAST($0):INTEGER, 5)], scanOrder=[FORWARD])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(true, new Object[][] {
 +                        {1, 2, 3, 4},
 +                        {2, 3, 4, 5}})
 +                .close();
 +        start(true).sql("select * from " + SALTED_TABLE_NAME + " s1, " + SALTED_TABLE_NAME + " s2 where s1.mypk0 = s2.mypk0 and s1.mypk1 = s2.mypk1 and s1.mypk0 > 1 and s2.col1 < 6")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixServerJoin(condition=[AND(=($0, $4), =($1, $5))], joinType=[inner])\n" +
 +                           "    PhoenixTableScan(table=[[phoenix, SALTED_TEST_TABLE]], filter=[>($0, 1)])\n" +
 +                           "    PhoenixServerProject(MYPK0=[$1], MYPK1=[$2], COL0=[$3], COL1=[CAST($0):INTEGER])\n" +
 +                           "      PhoenixTableScan(table=[[phoenix, IDXSALTED_SALTED_TEST_TABLE]], filter=[<(CAST($0):INTEGER, 6)])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                        {2, 3, 4, 5, 2, 3, 4, 5}})
 +                .close();
 +    }
 +    
 +    @Test public void testMultiTenant() throws Exception {
 +        Properties props = getConnectionProps(true);
 +        start(props).sql("select * from " + MULTI_TENANT_TABLE)
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixTableScan(table=[[phoenix, MULTITENANT_TEST_TABLE]])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                        {"10", "2", 3, 4, 5},
 +                        {"15", "3", 4, 5, 6},
 +                        {"20", "4", 5, 6, 7},
 +                        {"20", "5", 6, 7, 8}})
 +                .close();
 +        
 +        start(props).sql("select * from " + MULTI_TENANT_TABLE + " where tenant_id = '20' and col1 > 1")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixServerProject(TENANT_ID=[$0], ID=[$2], COL0=[$3], COL1=[CAST($1):INTEGER], COL2=[$4])\n" +
 +                           "    PhoenixTableScan(table=[[phoenix, IDX_MULTITENANT_TEST_TABLE]], filter=[AND(=(CAST($0):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\" NOT NULL, '20'), >(CAST($1):INTEGER, 1))])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                        {"20", "4", 5, 6, 7},
 +                        {"20", "5", 6, 7, 8}})
 +                .close();
 +        
 +        try {
 +            start(props).sql("select * from " + MULTI_TENANT_VIEW1)
 +                .explainIs("")
 +                .close();
 +            fail("Should have got SQLException.");
 +        } catch (SQLException e) {
 +        }
 +        
 +        props.setProperty("TenantId", "15");
 +        start(props).sql("select * from " + MULTI_TENANT_TABLE)
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixTableScan(table=[[phoenix, MULTITENANT_TEST_TABLE]])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                        {"3", 4, 5, 6}})
 +                .close();
 +        
 +        start(props).sql("select * from " + MULTI_TENANT_TABLE + " where col1 > 1")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixServerProject(ID=[$1], COL0=[$2], COL1=[CAST($0):INTEGER], COL2=[$3])\n" +
 +                           "    PhoenixTableScan(table=[[phoenix, IDX_MULTITENANT_TEST_TABLE]], filter=[>(CAST($0):INTEGER, 1)])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                        {"3", 4, 5, 6}})
 +                .close();
 +        
 +        try {
 +            start(props).sql("select * from " + MULTI_TENANT_VIEW1)
 +                .explainIs("")
 +                .close();
 +            fail("Should have got SQLException.");
 +        } catch (SQLException e) {
 +        }
 +
 +        props.setProperty("TenantId", "10");
 +        start(props).sql("select * from " + MULTI_TENANT_VIEW1)
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixTableScan(table=[[phoenix, MULTITENANT_TEST_TABLE]])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                        {"2", 3, 4, 5}})
 +                .close();
 +        
 +        start(props).sql("select * from " + MULTI_TENANT_TABLE + " where col1 > 1")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixServerProject(ID=[$1], COL0=[$2], COL1=[CAST($0):INTEGER], COL2=[$3])\n" +
 +                           "    PhoenixTableScan(table=[[phoenix, IDX_MULTITENANT_TEST_TABLE]], filter=[>(CAST($0):INTEGER, 1)])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                        {"2", 3, 4, 5}})
 +                .close();
 +        
 +        start(props).sql("select id, col0 from " + MULTI_TENANT_VIEW1 + " where col0 > 1")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixServerProject(ID=[$1], COL0=[CAST($0):INTEGER])\n" +
 +                           "    PhoenixTableScan(table=[[phoenix, S1, IDX_MULTITENANT_TEST_VIEW1]], filter=[>(CAST($0):INTEGER, 1)])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                        {"2", 3}})
 +                .close();
 +
 +        props.setProperty("TenantId", "20");
 +        start(props).sql("select * from " + MULTI_TENANT_VIEW2)
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixTableScan(table=[[phoenix, MULTITENANT_TEST_TABLE]], filter=[>($3, 7)])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                        {"5", 6, 7, 8}})
 +                .close();
 +        
 +        start(props).sql("select id, col0 from " + MULTI_TENANT_VIEW2 + " where col0 > 1")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixServerProject(ID=[$1], COL0=[CAST($0):INTEGER])\n" +
 +                           "    PhoenixTableScan(table=[[phoenix, S2, IDX_MULTITENANT_TEST_VIEW2]], filter=[>(CAST($0):INTEGER, 1)])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                        {"5", 6}})
 +                .close();
 +        
 +        start(props).sql("select id, col0 from " + MULTI_TENANT_VIEW2 + " order by col0")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixServerProject(ID=[$1], COL0=[CAST($0):INTEGER])\n" +
 +                           "    PhoenixTableScan(table=[[phoenix, S2, IDX_MULTITENANT_TEST_VIEW2]], scanOrder=[FORWARD])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(true, new Object[][] {
 +                        {"5", 6}})
 +                .close();
 +    }
 +
 +}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/74409be8/phoenix-core/src/main/java/org/apache/phoenix/calcite/CalciteRuntime.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/main/java/org/apache/phoenix/calcite/CalciteRuntime.java
index 52022a8,0000000..000a0c6
mode 100644,000000..100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/CalciteRuntime.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/CalciteRuntime.java
@@@ -1,148 -1,0 +1,156 @@@
 +package org.apache.phoenix.calcite;
 +
 +import org.apache.calcite.linq4j.AbstractEnumerable;
 +import org.apache.calcite.linq4j.Enumerable;
 +import org.apache.calcite.linq4j.Enumerator;
 +import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 +import org.apache.phoenix.compile.ColumnProjector;
 +import org.apache.phoenix.compile.QueryPlan;
 +import org.apache.phoenix.compile.RowProjector;
 +import org.apache.phoenix.iterate.ResultIterator;
 +import org.apache.phoenix.schema.tuple.Tuple;
 +import org.apache.phoenix.schema.types.PDataType;
 +import org.apache.phoenix.schema.types.PDate;
 +import org.apache.phoenix.schema.types.PDateArray;
++import org.apache.phoenix.schema.types.PDouble;
++import org.apache.phoenix.schema.types.PFloat;
 +import org.apache.phoenix.schema.types.PLong;
 +import org.apache.phoenix.schema.types.PLongArray;
 +import org.apache.phoenix.schema.types.PTime;
 +import org.apache.phoenix.schema.types.PTimeArray;
 +import org.apache.phoenix.schema.types.PTimestamp;
 +import org.apache.phoenix.schema.types.PTimestampArray;
 +import org.apache.phoenix.schema.types.PUnsignedDate;
 +import org.apache.phoenix.schema.types.PUnsignedDateArray;
++import org.apache.phoenix.schema.types.PUnsignedFloat;
 +import org.apache.phoenix.schema.types.PUnsignedTime;
 +import org.apache.phoenix.schema.types.PUnsignedTimeArray;
 +import org.apache.phoenix.schema.types.PUnsignedTimestamp;
 +import org.apache.phoenix.schema.types.PUnsignedTimestampArray;
 +import org.apache.phoenix.schema.types.PhoenixArray;
 +
 +import java.sql.SQLException;
 +import java.sql.Timestamp;
 +
 +/**
 + * Methods used by code generated by Calcite.
 + */
 +public class CalciteRuntime {
 +    public static Enumerable<Object> toEnumerable2(final ResultIterator iterator, final RowProjector rowProjector) {
 +        return new AbstractEnumerable<Object>() {
 +            @Override
 +            public Enumerator<Object> enumerator() {
 +                return toEnumerator(iterator, rowProjector);
 +            }
 +        };
 +    }
 +
 +    public static Enumerable<Object> toEnumerable(final QueryPlan plan) {
 +        try {
 +            return toEnumerable2(plan.iterator(), plan.getProjector());
 +        } catch (SQLException e) {
 +            throw new RuntimeException(e);
 +        }
 +    }
 +
 +    public static Enumerator<Object> toEnumerator(final ResultIterator iterator, final RowProjector rowProjector) {
 +        final int count = rowProjector.getColumnCount();
 +        return new Enumerator<Object>() {
 +            Object current;
 +            private final ImmutableBytesWritable ptr = new ImmutableBytesWritable();
 +
 +            @Override
 +            public Object current() {
 +                return current;
 +            }
 +
 +            @Override
 +            public boolean moveNext() {
 +                try {
 +                    final Tuple tuple = iterator.next();
 +                    if (tuple == null) {
 +                        current = null;
 +                        return false;
 +                    }
 +                    if (count == 1) {
 +                        ColumnProjector projector = rowProjector.getColumnProjector(0);
 +                        current = project(tuple, projector);
 +                        return true;
 +                    }
 +                    Object[] array = new Object[count];
 +                    for (int i = 0; i < count; i++) {
 +                        ColumnProjector projector = rowProjector.getColumnProjector(i);
 +                        array[i] = project(tuple, projector);
 +                    }
 +                    current = array;
 +                    return true;
 +                } catch (SQLException e) {
 +                    throw new RuntimeException(e);
 +                }
 +            }
 +            
 +            private Object project(Tuple tuple, ColumnProjector projector) throws SQLException {
 +                @SuppressWarnings("rawtypes")
 +                PDataType type = projector.getExpression().getDataType();
 +                if (PDataType.equalsAny(
 +                        type,
++                        PUnsignedFloat.INSTANCE,
++                        PFloat.INSTANCE)) {
++                    type = PDouble.INSTANCE;
++                } else if (PDataType.equalsAny(
++                        type,
 +                        PUnsignedDate.INSTANCE,
 +                        PDate.INSTANCE,
 +                        PUnsignedTime.INSTANCE,
 +                        PTime.INSTANCE)) {
 +                    type = PLong.INSTANCE;
-                 }else if (PDataType.equalsAny(
++                } else if (PDataType.equalsAny(
 +                        type,
 +                        PUnsignedDateArray.INSTANCE,
 +                        PDateArray.INSTANCE,
 +                        PUnsignedTimeArray.INSTANCE,
 +                        PTimeArray.INSTANCE)) {
 +                    type = PLongArray.INSTANCE;
 +                }
 +                Object value = projector.getValue(tuple, type, ptr);
 +                if (value != null) {
 +                    if (type.isArrayType()) {
 +                        value = ((PhoenixArray) value).getArray();
 +                    }
 +                    if (PDataType.equalsAny(
 +                                type,
 +                                PUnsignedTimestamp.INSTANCE,
 +                                PTimestamp.INSTANCE)) {
 +                        value = ((Timestamp) value).getTime();
 +                    } else if (PDataType.equalsAny(
 +                                type,
 +                                PUnsignedTimestampArray.INSTANCE,
 +                                PTimestampArray.INSTANCE)) {
 +                        Timestamp[] array = (Timestamp[]) value;
 +                        long[] newArray = new long[array.length];
 +                        for (int i = 0; i < array.length; i++) {
 +                            newArray[i] = array[i].getTime();
 +                        }
 +                        value = newArray;
 +                    }
 +                }
 +                
 +                return value;
 +            }
 +
 +            @Override
 +            public void reset() {
 +                throw new UnsupportedOperationException();
 +            }
 +
 +            @Override
 +            public void close() {
 +                try {
 +                    iterator.close();
 +                } catch (SQLException e) {
 +                    throw new RuntimeException(e);
 +                }
 +            }
 +        };
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/74409be8/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixSchema.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixSchema.java
index 8b2c160,0000000..fd630ab
mode 100644,000000..100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixSchema.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixSchema.java
@@@ -1,331 -1,0 +1,331 @@@
 +package org.apache.phoenix.calcite;
 +
 +import com.google.common.collect.ImmutableList;
 +import com.google.common.collect.Maps;
 +import com.google.common.collect.Sets;
 +
 +import org.apache.calcite.jdbc.CalciteSchema;
 +import org.apache.calcite.linq4j.tree.Expression;
 +import org.apache.calcite.materialize.MaterializationService;
 +import org.apache.calcite.schema.*;
 +import org.apache.calcite.schema.impl.ViewTable;
 +import org.apache.phoenix.compile.ColumnResolver;
 +import org.apache.phoenix.compile.FromCompiler;
 +import org.apache.phoenix.jdbc.PhoenixConnection;
 +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 +import org.apache.phoenix.parse.ColumnDef;
 +import org.apache.phoenix.parse.NamedTableNode;
 +import org.apache.phoenix.parse.TableName;
 +import org.apache.phoenix.schema.MetaDataClient;
 +import org.apache.phoenix.schema.PColumn;
 +import org.apache.phoenix.schema.PTable;
 +import org.apache.phoenix.schema.PTable.ViewType;
 +import org.apache.phoenix.schema.PTableImpl;
 +import org.apache.phoenix.schema.PTableType;
 +import org.apache.phoenix.schema.TableNotFoundException;
 +import org.apache.phoenix.schema.TableRef;
 +import org.apache.phoenix.util.IndexUtil;
 +import org.apache.phoenix.util.SchemaUtil;
 +
 +import java.sql.Connection;
 +import java.sql.DriverManager;
 +import java.sql.ResultSet;
 +import java.sql.SQLException;
 +import java.util.Collection;
 +import java.util.Collections;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Properties;
 +import java.util.Set;
 +
 +/**
 + * Implementation of Calcite's {@link Schema} SPI for Phoenix.
 + * 
 + * TODO
 + * 1) change this to non-caching mode??
 + * 2) how to deal with define indexes and views since they require a CalciteSchema
 + *    instance??
 + *
 + */
 +public class PhoenixSchema implements Schema {
 +    public static final Factory FACTORY = new Factory();
 +
 +    public final PhoenixConnection pc;
 +    
 +    protected final String name;
 +    protected final String schemaName;
 +    protected final SchemaPlus parentSchema;
 +    protected final MetaDataClient client;
 +    
 +    protected final Map<String, Schema> subSchemas;
 +    protected final Map<String, Table> tables;
 +    protected final Map<String, Function> views;
 +    protected final Set<PTable> viewTables;
 +    
 +    protected PhoenixSchema(String name, String schemaName,
 +            SchemaPlus parentSchema, PhoenixConnection pc) {
 +        this.name = name;
 +        this.schemaName = schemaName;
 +        this.parentSchema = parentSchema;
 +        this.pc = pc;
 +        this.client = new MetaDataClient(pc);
 +        this.subSchemas = Maps.newHashMap();
 +        this.tables = Maps.newHashMap();
 +        this.views = Maps.newHashMap();
 +        this.viewTables = Sets.newHashSet();
 +    }
 +
 +    private static Schema create(SchemaPlus parentSchema,
 +            String name, Map<String, Object> operand) {
 +        String url = (String) operand.get("url");
 +        final Properties properties = new Properties();
 +        for (Map.Entry<String, Object> entry : operand.entrySet()) {
 +            properties.setProperty(entry.getKey(), String.valueOf(entry.getValue()));
 +        }
 +        try {
 +            Class.forName("org.apache.phoenix.jdbc.PhoenixDriver");
 +            final Connection connection =
 +                DriverManager.getConnection(url, properties);
 +            final PhoenixConnection phoenixConnection =
 +                connection.unwrap(PhoenixConnection.class);
 +            return new PhoenixSchema(name, null, parentSchema, phoenixConnection);
 +        } catch (ClassNotFoundException e) {
 +            throw new RuntimeException(e);
 +        } catch (SQLException e) {
 +            throw new RuntimeException(e);
 +        }
 +    }
 +
 +    @Override
 +    public Table getTable(String name) {
 +        Table table = tables.get(name);
 +        if (table != null) {
 +            return table;
 +        }
 +        
 +        try {
 +            ColumnResolver x = FromCompiler.getResolver(
 +                    NamedTableNode.create(
 +                            null,
 +                            TableName.create(schemaName, name),
 +                            ImmutableList.<ColumnDef>of()), pc);
 +            final List<TableRef> tables = x.getTables();
 +            assert tables.size() == 1;
 +            PTable pTable = tables.get(0).getTable();
 +            if (!isView(pTable)) {
 +                pTable = fixTableMultiTenancy(pTable);
 +                table = new PhoenixTable(pc, pTable);
 +            }
 +        } catch (TableNotFoundException e) {
 +        } catch (SQLException e) {
 +            throw new RuntimeException(e);
 +        }
 +        
 +        if (table == null) {
 +            table = resolveSequence(name);
 +        }
 +        
 +        if (table != null) {
 +            tables.put(name, table);
 +        }
 +        return table;
 +    }
 +
 +    @Override
 +    public Set<String> getTableNames() {
 +        return tables.keySet();
 +    }
 +
 +    @Override
 +    public Collection<Function> getFunctions(String name) {
 +        Function func = views.get(name);
 +        if (func != null) {
 +            return ImmutableList.of(func);
 +        }
 +        
 +        try {
 +            ColumnResolver x = FromCompiler.getResolver(
 +                    NamedTableNode.create(
 +                            null,
 +                            TableName.create(schemaName, name),
 +                            ImmutableList.<ColumnDef>of()), pc);
 +            final List<TableRef> tables = x.getTables();
 +            assert tables.size() == 1;
 +            PTable pTable = tables.get(0).getTable();
 +            if (isView(pTable)) {
 +                String viewSql = pTable.getViewStatement();
 +                if (viewSql == null) {
 +                    viewSql = "select * from "
 +                            + SchemaUtil.getEscapedFullTableName(
 +                                    pTable.getPhysicalName().getString());
 +                }
 +                SchemaPlus schema = parentSchema.getSubSchema(this.name);
 +                SchemaPlus viewSqlSchema =
 +                        this.schemaName == null ? schema : parentSchema;
 +                func = ViewTable.viewMacro(schema, viewSql,
 +                        CalciteSchema.from(viewSqlSchema).path(null),
 +                        pTable.getViewType() == ViewType.UPDATABLE);
 +                views.put(name, func);
 +                viewTables.add(pTable);
 +            }
 +        } catch (TableNotFoundException e) {
 +        } catch (SQLException e) {
 +            throw new RuntimeException(e);
 +        }
 +        
 +        return func == null ? Collections.<Function>emptyList() : ImmutableList.of(func);
 +    }
 +
 +    @Override
 +    public Set<String> getFunctionNames() {
 +        return views.keySet();
 +    }
 +
 +    @Override
 +    public Schema getSubSchema(String name) {
 +        if (schemaName != null) {
 +            return null;
 +        }
 +        
 +        Schema schema = subSchemas.get(name);
 +        if (schema != null) {
 +            return schema;
 +        }
 +        
 +        schema = new PhoenixSchema(name, name, parentSchema.getSubSchema(this.name), pc);
 +        subSchemas.put(name, schema);
 +        return schema;
 +    }
 +
 +    @Override
 +    public Set<String> getSubSchemaNames() {
 +        return subSchemas.keySet();
 +    }
 +
 +    @Override
 +    public Expression getExpression(SchemaPlus parentSchema, String name) {
 +        throw new UnsupportedOperationException();
 +    }
 +
 +    @Override
 +    public boolean isMutable() {
 +        return true;
 +    }
 +
 +    @Override
 +    public boolean contentsHaveChangedSince(long lastCheck, long now) {
 +        return false;
 +    }
 +    
 +    public void defineIndexesAsMaterializations() {
 +        SchemaPlus schema = parentSchema.getSubSchema(this.name);
 +        SchemaPlus viewSqlSchema =
 +                this.schemaName == null ? schema : parentSchema;
 +        CalciteSchema calciteSchema = CalciteSchema.from(schema);
 +        List<String> path = CalciteSchema.from(viewSqlSchema).path(null);
 +        try {
 +            for (Table table : tables.values()) {
 +                if (table instanceof PhoenixTable) {
 +                    PTable pTable = ((PhoenixTable) table).pTable;
 +                    for (PTable index : pTable.getIndexes()) {
 +                        addMaterialization(index, path, calciteSchema);
 +                    }
 +                }
 +            }
 +            for (PTable pTable : viewTables) {
 +                for (PTable index : pTable.getIndexes()) {
 +                    if (index.getParentName().equals(pTable.getName())) {
 +                        addMaterialization(index, path, calciteSchema);
 +                    }
 +                }                
 +            }
 +        } catch (SQLException e) {
 +            throw new RuntimeException(e);
 +        }
 +    }
 +    
 +    private void addMaterialization(PTable index, List<String> path,
 +            CalciteSchema calciteSchema) throws SQLException {
 +        index = fixTableMultiTenancy(index);
 +        StringBuffer sb = new StringBuffer();
 +        sb.append("SELECT");
 +        for (PColumn column : PhoenixTable.getMappedColumns(index)) {
 +            String indexColumnName = column.getName().getString();
 +            String dataColumnName = IndexUtil.getDataColumnName(indexColumnName);
 +            sb.append(",").append(SchemaUtil.getEscapedFullColumnName(dataColumnName));
 +            sb.append(" ").append(SchemaUtil.getEscapedFullColumnName(indexColumnName));
 +        }
 +        sb.setCharAt(6, ' '); // replace first comma with space.
 +        sb.append(" FROM ").append(SchemaUtil.getEscapedFullTableName(index.getParentName().getString()));
 +        MaterializationService.instance().defineMaterialization(
 +                calciteSchema, null, sb.toString(), path, index.getTableName().getString(), true, true);        
 +    }
 +    
 +    private boolean isView(PTable table) {
 +        return table.getType() == PTableType.VIEW
 +                && table.getViewType() != ViewType.MAPPED;
 +    }
 +    
 +    private PTable fixTableMultiTenancy(PTable table) throws SQLException {
 +        if (pc.getTenantId() != null || !table.isMultiTenant()) {
 +            return table;
 +        }
 +        return PTableImpl.makePTable(
 +                table.getTenantId(), table.getSchemaName(), table.getTableName(), table.getType(), table.getIndexState(), table.getTimeStamp(),
 +                table.getSequenceNumber(), table.getPKName(), table.getBucketNum(), PTableImpl.getColumnsToClone(table), table.getParentSchemaName(), table.getParentTableName(),
 +                table.getIndexes(), table.isImmutableRows(), table.getPhysicalNames(), table.getDefaultFamilyName(), table.getViewStatement(),
 +                table.isWALDisabled(), false, table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), table.getIndexType(),
-                 table.getTableStats(), table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.isTransactional());
++                table.rowKeyOrderOptimizable(), table.isTransactional(), table.getUpdateCacheFrequency(), table.getTableStats(), table.getBaseColumnCount());
 +    }
 +    
 +    private PhoenixSequence resolveSequence(String name) {
 +        try {
 +            // FIXME: Do this the same way as resolving a table after PHOENIX-2489.
 +            String tenantId = pc.getTenantId() == null ? null : pc.getTenantId().getString();
 +            String q = "select 1 from " + PhoenixDatabaseMetaData.SEQUENCE_FULLNAME_ESCAPED
 +                    + " where " + PhoenixDatabaseMetaData.SEQUENCE_SCHEMA
 +                    + (schemaName == null ? " is null" : " = '" + schemaName + "'")
 +                    + " and " + PhoenixDatabaseMetaData.SEQUENCE_NAME
 +                    + " = '" + name + "'"
 +                    + " and " + PhoenixDatabaseMetaData.TENANT_ID
 +                    + (tenantId == null ? " is null" : " = '" + tenantId + "'");
 +            ResultSet rs = pc.createStatement().executeQuery(q);
 +            if (rs.next()) {
 +                return new PhoenixSequence(schemaName, name, pc);
 +            }
 +        } catch (SQLException e) {
 +            throw new RuntimeException(e);
 +        }
 +        
 +        return null;
 +    }
 +
 +    /** Schema factory that creates a
 +     * {@link org.apache.phoenix.calcite.PhoenixSchema}.
 +     * This allows you to create a Phoenix schema inside a model.json file.
 +     *
 +     * <pre>{@code
 +     * {
 +     *   version: '1.0',
 +     *   defaultSchema: 'HR',
 +     *   schemas: [
 +     *     {
 +     *       name: 'HR',
 +     *       type: 'custom',
 +     *       factory: 'org.apache.phoenix.calcite.PhoenixSchema.Factory',
 +     *       operand: {
 +     *         url: "jdbc:phoenix:localhost",
 +     *         user: "scott",
 +     *         password: "tiger"
 +     *       }
 +     *     }
 +     *   ]
 +     * }
 +     * }</pre>
 +     */
 +    public static class Factory implements SchemaFactory {
 +        public Schema create(SchemaPlus parentSchema, String name, Map<String, Object> operand) {
 +            return PhoenixSchema.create(parentSchema, name, operand);
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/74409be8/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixRelImplementorImpl.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixRelImplementorImpl.java
index 44f3031,0000000..e1afc14
mode 100644,000000..100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixRelImplementorImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/rel/PhoenixRelImplementorImpl.java
@@@ -1,191 -1,0 +1,191 @@@
 +package org.apache.phoenix.calcite.rel;
 +
 +import java.sql.SQLException;
 +import java.util.Collections;
 +import java.util.List;
 +import java.util.Stack;
 +
 +import org.apache.hadoop.hbase.HConstants;
 +import org.apache.phoenix.calcite.CalciteUtils;
 +import org.apache.phoenix.calcite.PhoenixSequence;
 +import org.apache.phoenix.calcite.PhoenixTable;
 +import org.apache.phoenix.calcite.rel.PhoenixRel.ImplementorContext;
 +import org.apache.phoenix.compile.ColumnProjector;
 +import org.apache.phoenix.compile.ExpressionProjector;
 +import org.apache.phoenix.compile.QueryPlan;
 +import org.apache.phoenix.compile.RowProjector;
 +import org.apache.phoenix.compile.SequenceManager;
 +import org.apache.phoenix.compile.SequenceValueExpression;
 +import org.apache.phoenix.compile.TupleProjectionCompiler;
 +import org.apache.phoenix.coprocessor.MetaDataProtocol;
 +import org.apache.phoenix.execute.RuntimeContext;
 +import org.apache.phoenix.execute.TupleProjector;
 +import org.apache.phoenix.expression.ColumnExpression;
 +import org.apache.phoenix.expression.CorrelateVariableFieldAccessExpression;
 +import org.apache.phoenix.expression.Expression;
 +import org.apache.phoenix.parse.ParseNodeFactory;
 +import org.apache.phoenix.parse.SequenceValueParseNode;
 +import org.apache.phoenix.parse.TableName;
 +import org.apache.phoenix.schema.ColumnRef;
 +import org.apache.phoenix.schema.KeyValueSchema;
 +import org.apache.phoenix.schema.PColumn;
 +import org.apache.phoenix.schema.PColumnImpl;
 +import org.apache.phoenix.schema.PName;
 +import org.apache.phoenix.schema.PNameFactory;
 +import org.apache.phoenix.schema.PTable;
 +import org.apache.phoenix.schema.PTableImpl;
 +import org.apache.phoenix.schema.PTableType;
 +import org.apache.phoenix.schema.TableRef;
 +import org.apache.phoenix.schema.KeyValueSchema.KeyValueSchemaBuilder;
 +import org.apache.phoenix.schema.types.PDataType;
 +import org.apache.phoenix.util.SchemaUtil;
 +
 +import com.google.common.collect.Lists;
 +
 +public class PhoenixRelImplementorImpl implements PhoenixRel.Implementor {
 +    private final RuntimeContext runtimeContext;
 +	private TableRef tableRef;
 +	private List<PColumn> mappedColumns;
 +	private Stack<ImplementorContext> contextStack;
 +	private SequenceManager sequenceManager;
 +	
 +	public PhoenixRelImplementorImpl(RuntimeContext runtimeContext) {
 +	    this.runtimeContext = runtimeContext;
 +	    this.contextStack = new Stack<ImplementorContext>();
 +	}
 +	
 +    @Override
 +    public QueryPlan visitInput(int i, PhoenixRel input) {
 +        return input.implement(this);
 +    }
 +
 +	@Override
 +	public ColumnExpression newColumnExpression(int index) {
 +		ColumnRef colRef = new ColumnRef(this.tableRef, this.mappedColumns.get(index).getPosition());
 +		return colRef.newColumnExpression();
 +	}
 +    
 +    @SuppressWarnings("rawtypes")
 +    @Override
 +    public Expression newFieldAccessExpression(String variableId, int index, PDataType type) {
 +        Expression fieldAccessExpr = runtimeContext.newCorrelateVariableReference(variableId, index);
 +        return new CorrelateVariableFieldAccessExpression(runtimeContext, variableId, fieldAccessExpr);
 +    }
 +    
 +    @Override
 +    public SequenceValueExpression newSequenceExpression(PhoenixSequence seq, SequenceValueParseNode.Op op) {
 +        PName tenantName = seq.pc.getTenantId();
 +        TableName tableName = TableName.create(seq.schemaName, seq.sequenceName);
 +        try {
 +            return sequenceManager.newSequenceReference(tenantName, tableName, null, op);
 +        } catch (SQLException e) {
 +            throw new RuntimeException(e);
 +        }
 +    }
 +    
 +    @Override
 +    public RuntimeContext getRuntimeContext() {
 +        return runtimeContext;
 +    }
 +
 +    @Override
 +	public void setTableRef(TableRef tableRef) {
 +		this.tableRef = tableRef;
 +		this.mappedColumns = PhoenixTable.getMappedColumns(tableRef.getTable());
 +	}
 +    
 +    @Override
 +    public TableRef getTableRef() {
 +        return this.tableRef;
 +    }
 +    
 +    @Override
 +    public void setSequenceManager(SequenceManager sequenceManager) {
 +        this.sequenceManager = sequenceManager;
 +    }
 +
 +    @Override
 +    public void pushContext(ImplementorContext context) {
 +        this.contextStack.push(context);
 +    }
 +
 +    @Override
 +    public ImplementorContext popContext() {
 +        return contextStack.pop();
 +    }
 +
 +    @Override
 +    public ImplementorContext getCurrentContext() {
 +        return contextStack.peek();
 +    }
 +    
 +    @Override
 +    public PTable createProjectedTable() {
 +        List<ColumnRef> sourceColumnRefs = Lists.<ColumnRef> newArrayList();
 +        List<PColumn> columns = getCurrentContext().retainPKColumns ?
 +                  getTableRef().getTable().getColumns() : mappedColumns;
 +        for (PColumn column : columns) {
 +            sourceColumnRefs.add(new ColumnRef(getTableRef(), column.getPosition()));
 +        }
 +        
 +        try {
 +            return TupleProjectionCompiler.createProjectedTable(getTableRef(), sourceColumnRefs, getCurrentContext().retainPKColumns);
 +        } catch (SQLException e) {
 +            throw new RuntimeException(e);
 +        }
 +    }
 +    
 +    @Override
 +    public TupleProjector createTupleProjector() {
 +        KeyValueSchemaBuilder builder = new KeyValueSchemaBuilder(0);
 +        List<Expression> exprs = Lists.<Expression> newArrayList();
 +        for (PColumn column : mappedColumns) {
 +            if (!SchemaUtil.isPKColumn(column) || !getCurrentContext().retainPKColumns) {
 +                Expression expr = new ColumnRef(tableRef, column.getPosition()).newColumnExpression();
 +                exprs.add(expr);
 +                builder.addField(expr);                
 +            }
 +        }
 +        
 +        return new TupleProjector(builder.build(), exprs.toArray(new Expression[exprs.size()]));
 +    }
 +    
 +    @Override
 +    public RowProjector createRowProjector() {
 +        List<ColumnProjector> columnProjectors = Lists.<ColumnProjector>newArrayList();
 +        for (int i = 0; i < mappedColumns.size(); i++) {
 +            PColumn column = mappedColumns.get(i);
 +            Expression expr = newColumnExpression(i); // Do not use column.position() here.
 +            columnProjectors.add(new ExpressionProjector(column.getName().getString(), getTableRef().getTable().getName().getString(), expr, false));
 +        }
 +        // TODO get estimate row size
 +        return new RowProjector(columnProjectors, 0, false);        
 +    }
 +    
 +    @Override
 +    public TupleProjector project(List<Expression> exprs) {
 +        KeyValueSchema.KeyValueSchemaBuilder builder = new KeyValueSchema.KeyValueSchemaBuilder(0);
 +        List<PColumn> columns = Lists.<PColumn>newArrayList();
 +        for (int i = 0; i < exprs.size(); i++) {
 +            String name = ParseNodeFactory.createTempAlias();
 +            Expression expr = exprs.get(i);
 +            builder.addField(expr);
 +            columns.add(new PColumnImpl(PNameFactory.newName(name), PNameFactory.newName(TupleProjector.VALUE_COLUMN_FAMILY),
 +                    expr.getDataType(), expr.getMaxLength(), expr.getScale(), expr.isNullable(),
-                     i, expr.getSortOrder(), null, null, false, name, false));
++                    i, expr.getSortOrder(), null, null, false, name, false, false));
 +        }
 +        try {
 +            PTable pTable = PTableImpl.makePTable(null, PName.EMPTY_NAME, PName.EMPTY_NAME,
 +                    PTableType.SUBQUERY, null, MetaDataProtocol.MIN_TABLE_TIMESTAMP, PTable.INITIAL_SEQ_NUM,
 +                    null, null, columns, null, null, Collections.<PTable>emptyList(),
 +                    false, Collections.<PName>emptyList(), null, null, false, false, false, null,
-                     null, null, true, false);
++                    null, null, true, false, 0);
 +            this.setTableRef(new TableRef(CalciteUtils.createTempAlias(), pTable, HConstants.LATEST_TIMESTAMP, false));
 +        } catch (SQLException e) {
 +            throw new RuntimeException(e);
 +        }
 +        
 +        return new TupleProjector(builder.build(), exprs.toArray(new Expression[exprs.size()]));        
 +    }
 +
 +}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/74409be8/phoenix-core/src/main/java/org/apache/phoenix/compile/ListJarsQueryPlan.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/74409be8/phoenix-core/src/main/java/org/apache/phoenix/compile/TraceQueryPlan.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/74409be8/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientAggregatePlan.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/74409be8/phoenix-core/src/main/java/org/apache/phoenix/execute/HashJoinPlan.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/main/java/org/apache/phoenix/execute/HashJoinPlan.java
index 64a9140,5fdec46..e7a00ed
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/HashJoinPlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/HashJoinPlan.java
@@@ -450,16 -443,6 +450,16 @@@ public class HashJoinPlan extends Deleg
              return plan;
          }
      }
 +
 +    @Override
 +    public QueryPlan limit(Integer limit) {
 +        QueryPlan delegate = this.delegate.limit(limit);
 +        if (delegate == this.delegate)
 +            return this;
 +
 +        return new HashJoinPlan(this.statement, delegate, this.joinInfo, 
-                 this.subPlans, this.recompileWhereClause);
++                this.subPlans, this.recompileWhereClause, dependencies);
 +    }
  }
  
  

http://git-wip-us.apache.org/repos/asf/phoenix/blob/74409be8/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
index b3235e2,b3235e2..51715f9
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
@@@ -143,7 -143,7 +143,7 @@@ public abstract class BaseResultIterato
          if (isPointLookup || ScanUtil.isAnalyzeTable(scan)) {
              return false;
          }
--        return true;
++        return false;
      }
      
      private static void initializeScan(QueryPlan plan, Integer perScanLimit) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/74409be8/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/74409be8/pom.xml
----------------------------------------------------------------------
diff --cc pom.xml
index a1dab3c,4373753..fc642da
--- a/pom.xml
+++ b/pom.xml
@@@ -109,9 -109,12 +109,12 @@@
      <collections.version>3.2.1</collections.version>
      <jodatime.version>2.7</jodatime.version>
      <joni.version>2.1.2</joni.version>
 -    <calcite.version>1.6.0</calcite.version>
 +    <calcite.version>1.6.0-SNAPSHOT</calcite.version>
      <jettyVersion>8.1.7.v20120910</jettyVersion>
      <tephra.version>0.6.4</tephra.version>
+     <spark.version>1.5.2</spark.version>
+     <scala.version>2.10.4</scala.version>
+     <scala.binary.version>2.10</scala.binary.version>
  
      <!-- Test Dependencies -->
      <mockito-all.version>1.8.5</mockito-all.version>


[03/50] [abbrv] phoenix git commit: PHOENIX-2593 Ensure all built-ins have required constructors (Dumindu Buddhika)

Posted by ma...@apache.org.
PHOENIX-2593 Ensure all built-ins have required constructors (Dumindu Buddhika)


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

Branch: refs/heads/calcite
Commit: 9a44b492673497e65e41ff45f3bcf7fb23982e35
Parents: 59b336e
Author: James Taylor <jt...@salesforce.com>
Authored: Sun Jan 17 12:11:04 2016 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Sun Jan 17 12:11:04 2016 -0800

----------------------------------------------------------------------
 .../expression/function/CeilFunction.java       |  2 +
 .../expression/function/FloorFunction.java      |  4 +-
 .../expression/function/NowFunction.java        |  2 +
 .../expression/function/RoundFunction.java      |  4 +-
 .../expression/function/ToCharFunction.java     |  4 ++
 .../expression/function/ToDateFunction.java     |  4 ++
 .../expression/function/ToNumberFunction.java   |  4 ++
 .../expression/function/ToTimeFunction.java     |  4 ++
 .../function/ToTimestampFunction.java           |  4 ++
 .../expression/function/TruncFunction.java      |  2 +
 .../BuiltinFunctionConstructorTest.java         | 65 ++++++++++++++++++++
 11 files changed, 95 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/9a44b492/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CeilFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CeilFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CeilFunction.java
index 706aadf..b65010a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CeilFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CeilFunction.java
@@ -46,6 +46,8 @@ import org.apache.phoenix.schema.types.PVarchar;
 public abstract class CeilFunction extends ScalarFunction {
     
     public static final String NAME = "CEIL";
+
+    public CeilFunction() {}
     
     public CeilFunction(List<Expression> children) {
         super(children);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/9a44b492/phoenix-core/src/main/java/org/apache/phoenix/expression/function/FloorFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/FloorFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/FloorFunction.java
index 7938324..f6baa09 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/FloorFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/FloorFunction.java
@@ -32,8 +32,6 @@ import org.apache.phoenix.schema.types.PVarchar;
  * 
  * Base class for built-in FLOOR function.
  *
- * 
- * @since 3.0.0
  */
 @BuiltInFunction(name = FloorFunction.NAME,
                  nodeClass = FloorParseNode.class,
@@ -47,6 +45,8 @@ public abstract class FloorFunction extends ScalarFunction {
     
     public static final String NAME = "FLOOR";
     
+    public FloorFunction() {}
+    
     public FloorFunction(List<Expression> children) {
         super(children);
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/9a44b492/phoenix-core/src/main/java/org/apache/phoenix/expression/function/NowFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/NowFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/NowFunction.java
index dc90249..83a5e53 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/NowFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/NowFunction.java
@@ -35,6 +35,8 @@ nodeClass=CurrentDateParseNode.class, args= {})
 public abstract class NowFunction extends ScalarFunction {
     
     public static final String NAME = "NOW";
+
+    public NowFunction() {}
     
     public NowFunction(List<Expression> children) throws SQLException {
         super(children);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/9a44b492/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RoundFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RoundFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RoundFunction.java
index 9277716..7d0306c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RoundFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RoundFunction.java
@@ -31,8 +31,6 @@ import org.apache.phoenix.schema.types.PVarchar;
 /**
  * Base class for RoundFunction.
  * 
- * 
- * @since 0.1
  */
 @BuiltInFunction(name = RoundFunction.NAME, 
                  nodeClass = RoundParseNode.class,
@@ -46,6 +44,8 @@ public abstract class RoundFunction extends ScalarFunction {
     
     public static final String NAME = "ROUND";
     
+    public RoundFunction() {}
+    
     public RoundFunction(List<Expression> children) {
         super(children);
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/9a44b492/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToCharFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToCharFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToCharFunction.java
index e0bc68b..a14a0cb 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToCharFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToCharFunction.java
@@ -57,6 +57,10 @@ public class ToCharFunction extends ScalarFunction {
     public ToCharFunction() {
     }
 
+    public ToCharFunction(List<Expression> children) throws  SQLException {
+        this(children, null, null, null);
+    }
+
     public ToCharFunction(List<Expression> children, FunctionArgumentType type, String formatString, Format formatter) throws SQLException {
         super(children.subList(0, 1));
         Preconditions.checkNotNull(formatString);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/9a44b492/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToDateFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToDateFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToDateFunction.java
index 01b0dfd..f41d959 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToDateFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToDateFunction.java
@@ -59,6 +59,10 @@ public class ToDateFunction extends ScalarFunction {
     public ToDateFunction() {
     }
 
+    public ToDateFunction(List<Expression> children) throws SQLException {
+        this(children, null, null);
+    }
+    
     public ToDateFunction(List<Expression> children, String dateFormat, String timeZoneId) throws SQLException {
         super(children);
         init(dateFormat, timeZoneId);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/9a44b492/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToNumberFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToNumberFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToNumberFunction.java
index ed70f2d..e7659a1 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToNumberFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToNumberFunction.java
@@ -60,6 +60,10 @@ public class ToNumberFunction extends ScalarFunction {
     
     public ToNumberFunction() {}
 
+    public ToNumberFunction(List<Expression> children) throws SQLException {
+        this(children, null, null, null);
+    }
+
     public ToNumberFunction(List<Expression> children, FunctionArgumentType type, String formatString, Format formatter) throws SQLException {
         super(children.subList(0, 1));
         Preconditions.checkNotNull(type);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/9a44b492/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToTimeFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToTimeFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToTimeFunction.java
index 3a26584..2fa0553 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToTimeFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToTimeFunction.java
@@ -47,6 +47,10 @@ public class ToTimeFunction extends ToDateFunction {
     public ToTimeFunction() {
     }
 
+    public ToTimeFunction(List<Expression> children) throws SQLException {
+        this(children, null, null);
+    }
+
     public ToTimeFunction(List<Expression> children, String dateFormat, String timeZoneId) throws SQLException {
         super(children, dateFormat, timeZoneId);
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/9a44b492/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToTimestampFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToTimestampFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToTimestampFunction.java
index 17643a2..5a4828f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToTimestampFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ToTimestampFunction.java
@@ -47,6 +47,10 @@ public class ToTimestampFunction extends ToDateFunction {
     public ToTimestampFunction() {
     }
 
+    public ToTimestampFunction(List<Expression> children) throws SQLException {
+        this(children, null, null);
+    }
+
     public ToTimestampFunction(List<Expression> children, String dateFormat, String timeZoneId) throws SQLException {
         super(children, dateFormat, timeZoneId);
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/9a44b492/phoenix-core/src/main/java/org/apache/phoenix/expression/function/TruncFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/TruncFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/TruncFunction.java
index a4e79c2..3b9616c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/TruncFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/TruncFunction.java
@@ -51,6 +51,8 @@ args = {
 public abstract class TruncFunction extends ScalarFunction {
     
     public static final String NAME = "TRUNC";
+
+    public TruncFunction() {}
     
     public TruncFunction(List<Expression> children) throws SQLException {
         super(children);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/9a44b492/phoenix-core/src/test/java/org/apache/phoenix/expression/function/BuiltinFunctionConstructorTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/expression/function/BuiltinFunctionConstructorTest.java b/phoenix-core/src/test/java/org/apache/phoenix/expression/function/BuiltinFunctionConstructorTest.java
new file mode 100644
index 0000000..5258283
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/expression/function/BuiltinFunctionConstructorTest.java
@@ -0,0 +1,65 @@
+/*
+ * 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.phoenix.expression.function;
+
+import java.lang.reflect.Constructor;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.phoenix.expression.function.ScalarFunction;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.ExpressionType;
+import org.junit.Test;
+
+public class BuiltinFunctionConstructorTest {
+
+    @Test
+    public void testChildrenListConstructors() {
+        ExpressionType[] types = ExpressionType.values();
+        List<Expression> children = new ArrayList<>();
+        for(int i = 0; i < types.length; i++) {
+            try {
+                if((ScalarFunction.class.isAssignableFrom(types[i].getExpressionClass())) && (types[i].getExpressionClass() != UDFExpression.class)) {
+                    Constructor cons = types[i].getExpressionClass().getDeclaredConstructor(List.class);
+                    cons.setAccessible(true);
+                    cons.newInstance(children);
+                }
+            } catch (NoSuchMethodException e) {
+                throw new RuntimeException(e);
+            } catch (Exception e) {
+            }
+        }
+    }
+
+    @Test
+    public void testNoArgumentConstructors() {
+        ExpressionType[] types = ExpressionType.values();
+        for(int i = 0; i < types.length; i++) {
+            try {
+                if(!AggregateFunction.class.isAssignableFrom(types[i].getExpressionClass())) {
+                    Constructor cons = types[i].getExpressionClass().getDeclaredConstructor();
+                    cons.setAccessible(true);
+                    cons.newInstance();
+                }
+            } catch (NoSuchMethodException e) {
+                throw new RuntimeException(e);
+            } catch (Exception e) {
+            }
+        }
+    }
+}


[41/50] [abbrv] phoenix git commit: PHOENIX-2630 Ensure commit fails if mutable secondary index write fails

Posted by ma...@apache.org.
PHOENIX-2630 Ensure commit fails if mutable secondary index write fails


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

Branch: refs/heads/calcite
Commit: c9594e83e144ebc3830922401de2addb3df0c81d
Parents: dceaaeb
Author: James Taylor <jt...@salesforce.com>
Authored: Wed Jan 27 14:28:34 2016 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Thu Jan 28 10:15:01 2016 -0800

----------------------------------------------------------------------
 .../apache/phoenix/end2end/index/MutableIndexFailureIT.java | 6 +++++-
 .../hbase/index/write/KillServerOnFailurePolicy.java        | 5 ++---
 .../org/apache/phoenix/index/PhoenixIndexFailurePolicy.java | 9 +++++++--
 3 files changed, 14 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/c9594e83/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java
index b34a70e..5f39515 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java
@@ -193,7 +193,11 @@ public class MutableIndexFailureIT extends BaseOwnClusterHBaseManagedTimeIT {
                 }
             }
             else {
-                conn.commit();
+                try {
+                    conn.commit();
+                    fail();
+                } catch (SQLException e) {
+                }
             }
 
             // Verify the metadata for index is correct.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c9594e83/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/KillServerOnFailurePolicy.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/KillServerOnFailurePolicy.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/KillServerOnFailurePolicy.java
index 2fb43b5..cba2459 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/KillServerOnFailurePolicy.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/KillServerOnFailurePolicy.java
@@ -25,11 +25,10 @@ import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.phoenix.hbase.index.table.HTableInterfaceReference;
 
 import com.google.common.collect.Multimap;
 
-import org.apache.phoenix.hbase.index.table.HTableInterfaceReference;
-
 /**
  * Naive failure policy - kills the server on which it resides
  */
@@ -61,7 +60,7 @@ public class KillServerOnFailurePolicy implements IndexFailurePolicy {
 
   @Override
   public void
-      handleFailure(Multimap<HTableInterfaceReference, Mutation> attempted, Exception cause){
+      handleFailure(Multimap<HTableInterfaceReference, Mutation> attempted, Exception cause) throws IOException{
     // cleanup resources
     this.stop("Killing ourselves because of an error:" + cause);
     // notify the regionserver of the failure

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c9594e83/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java
index 806a20a..09a8676 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java
@@ -61,6 +61,7 @@ import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.ServerUtil;
 
 import com.google.common.collect.Multimap;
 
@@ -99,13 +100,17 @@ public class PhoenixIndexFailurePolicy extends KillServerOnFailurePolicy {
      * @param cause root cause of the failure
      */
     @Override
-    public void handleFailure(Multimap<HTableInterfaceReference, Mutation> attempted, Exception cause) {
-
+    public void handleFailure(Multimap<HTableInterfaceReference, Mutation> attempted, Exception cause) throws IOException {
+        boolean throwing = true;
         try {
             handleFailureWithExceptions(attempted, cause);
+            throwing = false;
         } catch (Throwable t) {
             LOG.warn("handleFailure failed", t);
             super.handleFailure(attempted, cause);
+            throwing = false;
+        } finally {
+            if (!throwing) throw ServerUtil.createIOException(null, cause);
         }
     }
 


[40/50] [abbrv] phoenix git commit: PHOENIX-2638 Move spark version properties main pom (Ankit Singhal)

Posted by ma...@apache.org.
PHOENIX-2638 Move spark version properties main pom (Ankit Singhal)


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

Branch: refs/heads/calcite
Commit: dceaaeb8aa6eb4213ca5868e03cf1cca16fa4d01
Parents: a6646d7
Author: Josh Mahonin <jm...@gmail.com>
Authored: Thu Jan 28 11:06:51 2016 -0500
Committer: Josh Mahonin <jm...@gmail.com>
Committed: Thu Jan 28 11:06:51 2016 -0500

----------------------------------------------------------------------
 phoenix-spark/pom.xml | 3 ---
 pom.xml               | 3 +++
 2 files changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/dceaaeb8/phoenix-spark/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-spark/pom.xml b/phoenix-spark/pom.xml
index 4091afa..79a029a 100644
--- a/phoenix-spark/pom.xml
+++ b/phoenix-spark/pom.xml
@@ -34,9 +34,6 @@
   <name>Phoenix - Spark</name>
 
   <properties>
-    <spark.version>1.5.2</spark.version>
-    <scala.version>2.10.4</scala.version>
-    <scala.binary.version>2.10</scala.binary.version>
     <top.dir>${project.basedir}/..</top.dir>
   </properties>
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/dceaaeb8/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 388df2e..4373753 100644
--- a/pom.xml
+++ b/pom.xml
@@ -112,6 +112,9 @@
     <calcite.version>1.6.0</calcite.version>
     <jettyVersion>8.1.7.v20120910</jettyVersion>
     <tephra.version>0.6.4</tephra.version>
+    <spark.version>1.5.2</spark.version>
+    <scala.version>2.10.4</scala.version>
+    <scala.binary.version>2.10</scala.binary.version>
 
     <!-- Test Dependencies -->
     <mockito-all.version>1.8.5</mockito-all.version>


[13/50] [abbrv] phoenix git commit: PHOENIX-2478 Rows committed in transaction overlapping index creation are not populated

Posted by ma...@apache.org.
PHOENIX-2478 Rows committed in transaction overlapping index creation are not populated


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

Branch: refs/heads/calcite
Commit: 4d323b66b8f685872c7b94f3d243c940f7fa37d9
Parents: b63ca5f
Author: James Taylor <jt...@salesforce.com>
Authored: Tue Jan 19 22:10:48 2016 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Tue Jan 19 22:11:00 2016 -0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/phoenix/execute/MutationState.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/4d323b66/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
index 3dfae46..f1a9c02 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
@@ -1234,7 +1234,7 @@ public class MutationState implements SQLCloseable {
             if (updatedDataTable == null) {
                 throw new TableNotFoundException(dataTable.getSchemaName().getString(), dataTable.getTableName().getString());
             }
-            allImmutableTables |= updatedDataTable.isImmutableRows();
+            allImmutableTables &= updatedDataTable.isImmutableRows();
             tableRef.setTable(updatedDataTable);
             if (!addedAnyIndexes) {
                 // TODO: in theory we should do a deep equals check here, as it's possible


[34/50] [abbrv] phoenix git commit: PHOENIX-2279 Error result when there are both GROUP-BY and LIMIT operation on derived table

Posted by ma...@apache.org.
PHOENIX-2279 Error result when there are both GROUP-BY and LIMIT operation on derived table


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

Branch: refs/heads/calcite
Commit: e2b6009f24b0b1019eb7cfaceea593556befb2cb
Parents: 551cc7d
Author: maryannxue <ma...@gmail.com>
Authored: Mon Jan 25 21:26:09 2016 -0500
Committer: maryannxue <ma...@gmail.com>
Committed: Mon Jan 25 21:26:09 2016 -0500

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/DerivedTableIT.java     | 13 +++++++++++++
 .../apache/phoenix/execute/ClientAggregatePlan.java    |  2 +-
 2 files changed, 14 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/e2b6009f/phoenix-core/src/it/java/org/apache/phoenix/end2end/DerivedTableIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DerivedTableIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DerivedTableIT.java
index 223a0e5..151ce3e 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DerivedTableIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DerivedTableIT.java
@@ -528,6 +528,19 @@ public class DerivedTableIT extends BaseClientManagedTimeIT {
             assertEquals(10,rs.getInt(2));
 
             assertFalse(rs.next());
+            
+            // (union) groupby limit
+            query = "SELECT a_string, count(*) FROM (SELECT a_string FROM aTable where a_byte < 4 union all SELECT a_string FROM aTable where a_byte > 8) group by a_string limit 2";
+            statement = conn.prepareStatement(query);
+            rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(A_VALUE,rs.getString(1));
+            assertEquals(3,rs.getInt(2));
+            assertTrue (rs.next());
+            assertEquals(C_VALUE,rs.getString(1));
+            assertEquals(1,rs.getInt(2));
+
+            assertFalse(rs.next());            
         } finally {
             conn.close();
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e2b6009f/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientAggregatePlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientAggregatePlan.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientAggregatePlan.java
index 3df0447..c1ef67d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientAggregatePlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientAggregatePlan.java
@@ -100,7 +100,7 @@ public class ClientAggregatePlan extends ClientProcessingPlan {
                 for (Expression keyExpression : keyExpressions) {
                     keyExpressionOrderBy.add(new OrderByExpression(keyExpression, false, true));
                 }
-                iterator = new OrderedResultIterator(iterator, keyExpressionOrderBy, thresholdBytes, limit, projector.getEstimatedRowByteSize());
+                iterator = new OrderedResultIterator(iterator, keyExpressionOrderBy, thresholdBytes, null, projector.getEstimatedRowByteSize());
             }
             aggResultIterator = new ClientGroupedAggregatingResultIterator(LookAheadResultIterator.wrap(iterator), serverAggregators, groupBy.getKeyExpressions());
             aggResultIterator = new GroupedAggregatingResultIterator(LookAheadResultIterator.wrap(aggResultIterator), clientAggregators);


[08/50] [abbrv] phoenix git commit: PHOENIX-2478 Rows committed in transaction overlapping index creation are not populated

Posted by ma...@apache.org.
PHOENIX-2478 Rows committed in transaction overlapping index creation are not populated


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

Branch: refs/heads/calcite
Commit: f572fa630a5bdfe5f1eab074838a82344d38063d
Parents: 1717f12
Author: James Taylor <jt...@salesforce.com>
Authored: Tue Jan 19 20:23:51 2016 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Tue Jan 19 20:44:47 2016 -0800

----------------------------------------------------------------------
 .../phoenix/end2end/index/ImmutableIndexIT.java |   6 +-
 .../end2end/index/IndexExpressionIT.java        |  16 +--
 .../phoenix/compile/PostIndexDDLCompiler.java   |   2 +-
 .../phoenix/exception/SQLExceptionCode.java     | 108 +++++++++----------
 .../apache/phoenix/execute/MutationState.java   |  39 ++++---
 .../apache/phoenix/jdbc/PhoenixStatement.java   |   2 +-
 .../apache/phoenix/schema/MetaDataClient.java   |   2 +-
 .../org/apache/phoenix/schema/TableRef.java     |   8 +-
 .../phoenix/compile/QueryCompilerTest.java      |   2 +-
 9 files changed, 98 insertions(+), 87 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/f572fa63/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
index c4ecfbb..7171382 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
@@ -219,7 +219,7 @@ public class ImmutableIndexIT extends BaseHBaseManagedTimeIT {
                 futureList.add(threadPool.submit(new UpsertRunnable(fullTableName)));
             }
             // upsert some rows before creating the index 
-            Thread.sleep(5000);
+            Thread.sleep(500);
 
             // create the index 
             try (Connection conn2 = DriverManager.getConnection(getUrl(), props)) {
@@ -230,14 +230,14 @@ public class ImmutableIndexIT extends BaseHBaseManagedTimeIT {
             }
 
             // upsert some rows after creating the index
-            Thread.sleep(1000);
+            Thread.sleep(100);
             // cancel the running threads
             for (Future<?> future : futureList) {
                 future.cancel(true);
             }
             threadPool.shutdownNow();
             threadPool.awaitTermination(30, TimeUnit.SECONDS);
-            Thread.sleep(1000);
+            Thread.sleep(100);
 
             ResultSet rs;
             rs = conn.createStatement().executeQuery("SELECT /*+ NO_INDEX */ COUNT(*) FROM " + fullTableName);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f572fa63/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexExpressionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexExpressionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexExpressionIT.java
index 7da0d85..7be8d41 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexExpressionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexExpressionIT.java
@@ -95,7 +95,7 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
 
     private void verifyResult(ResultSet rs, int i) throws SQLException {
         assertTrue(rs.next());
-        assertEquals("VARCHAR" + String.valueOf(i) + "_" + StringUtils.rightPad("CHAR" + String.valueOf(i), 6, ' ')
+        assertEquals("VARCHAR" + String.valueOf(i) + "_" + StringUtils.rightPad("CHAR" + String.valueOf(i), 10, ' ')
                 + "_A.VARCHAR" + String.valueOf(i) + "_" + StringUtils.rightPad("B.CHAR" + String.valueOf(i), 10, ' '),
                 rs.getString(1));
         assertEquals(i * 3, rs.getInt(2));
@@ -141,7 +141,7 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
                     // DECIMAL in the index (which is not fixed width)
                     + " AND date_pk+1=? AND date1+1=? AND date2+1=?";
             stmt = conn.prepareStatement(whereSql);
-            stmt.setString(1, "VARCHAR1_CHAR1 _A.VARCHAR1_B.CHAR1   ");
+            stmt.setString(1, "VARCHAR1_CHAR1     _A.VARCHAR1_B.CHAR1   ");
             stmt.setInt(2, 3);
             Date date = DateUtil.parseDate("2015-01-02 00:00:00");
             stmt.setDate(3, date);
@@ -153,8 +153,8 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
             assertEquals(
                     localIndex ? "CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_INDEX_TEST."
                             + dataTableName
-                            + " [-32768,'VARCHAR1_CHAR1 _A.VARCHAR1_B.CHAR1   ',3,'2015-01-02 00:00:00.000',1,420,156,800,000,1,420,156,800,000]\nCLIENT MERGE SORT"
-                            : "CLIENT PARALLEL 1-WAY RANGE SCAN OVER INDEX_TEST.IDX ['VARCHAR1_CHAR1 _A.VARCHAR1_B.CHAR1   ',3,'2015-01-02 00:00:00.000',1,420,156,800,000,1,420,156,800,000]",
+                            + " [-32768,'VARCHAR1_CHAR1     _A.VARCHAR1_B.CHAR1   ',3,'2015-01-02 00:00:00.000',1,420,156,800,000,1,420,156,800,000]\nCLIENT MERGE SORT"
+                            : "CLIENT PARALLEL 1-WAY RANGE SCAN OVER INDEX_TEST.IDX ['VARCHAR1_CHAR1     _A.VARCHAR1_B.CHAR1   ',3,'2015-01-02 00:00:00.000',1,420,156,800,000,1,420,156,800,000]",
                     QueryUtil.getExplainPlan(rs));
 
             // verify that the correct results are returned
@@ -254,20 +254,20 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
                     + fullDataTableName;
             ResultSet rs = conn.createStatement().executeQuery("SELECT /*+ NO_INDEX */ " + selectSql);
             assertTrue(rs.next());
-            assertEquals("VARCHAR1_CHAR1 _A.VARCHAR_UPDATED_B.CHAR1   ", rs.getString(1));
+            assertEquals("VARCHAR1_CHAR1     _A.VARCHAR_UPDATED_B.CHAR1   ", rs.getString(1));
             assertEquals(101, rs.getLong(2));
             assertTrue(rs.next());
-            assertEquals("VARCHAR2_CHAR2 _A.VARCHAR2_B.CHAR2   ", rs.getString(1));
+            assertEquals("VARCHAR2_CHAR2     _A.VARCHAR2_B.CHAR2   ", rs.getString(1));
             assertEquals(2, rs.getLong(2));
             assertFalse(rs.next());
 
             // verify that the rows in the index table are also updated
             rs = conn.createStatement().executeQuery("SELECT " + selectSql);
             assertTrue(rs.next());
-            assertEquals("VARCHAR1_CHAR1 _A.VARCHAR_UPDATED_B.CHAR1   ", rs.getString(1));
+            assertEquals("VARCHAR1_CHAR1     _A.VARCHAR_UPDATED_B.CHAR1   ", rs.getString(1));
             assertEquals(101, rs.getLong(2));
             assertTrue(rs.next());
-            assertEquals("VARCHAR2_CHAR2 _A.VARCHAR2_B.CHAR2   ", rs.getString(1));
+            assertEquals("VARCHAR2_CHAR2     _A.VARCHAR2_B.CHAR2   ", rs.getString(1));
             assertEquals(2, rs.getLong(2));
             assertFalse(rs.next());
             conn.createStatement().execute("DROP INDEX IDX ON " + fullDataTableName);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f572fa63/phoenix-core/src/main/java/org/apache/phoenix/compile/PostIndexDDLCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/PostIndexDDLCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/PostIndexDDLCompiler.java
index bb0b595..1a667ae 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/PostIndexDDLCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/PostIndexDDLCompiler.java
@@ -120,7 +120,7 @@ public class PostIndexDDLCompiler {
             DelegateMutationPlan delegate = new DelegateMutationPlan(statement.compileMutation(updateStmtStr.toString())) {
                 @Override
                 public MutationState execute() throws SQLException {
-                    connection.getMutationState().commitWriteFence(dataTable);
+                    connection.getMutationState().commitDDLFence(dataTable);
                     return super.execute();
                 }
             };

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f572fa63/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
index 9767cbe..b1d8e7d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
@@ -131,53 +131,53 @@ public enum SQLExceptionCode {
     PRIMARY_KEY_MISSING(509, "42888", "The table does not have a primary key."),
     PRIMARY_KEY_ALREADY_EXISTS(510, "42889", "The table already has a primary key."),
     ORDER_BY_NOT_IN_SELECT_DISTINCT(511, "42890", "All ORDER BY expressions must appear in SELECT DISTINCT:"),
-    INVALID_PRIMARY_KEY_CONSTRAINT(512, "42891", "Invalid column reference in primary key constraint"),
-    ARRAY_NOT_ALLOWED_IN_PRIMARY_KEY(513, "42892", "Array type not allowed as primary key constraint"),
+    INVALID_PRIMARY_KEY_CONSTRAINT(512, "42891", "Invalid column reference in primary key constraint."),
+    ARRAY_NOT_ALLOWED_IN_PRIMARY_KEY(513, "42892", "Array type not allowed as primary key constraint."),
     COLUMN_EXIST_IN_DEF(514, "42892", "A duplicate column name was detected in the object definition or ALTER TABLE statement.", new Factory() {
         @Override
         public SQLException newException(SQLExceptionInfo info) {
             return new ColumnAlreadyExistsException(info.getSchemaName(), info.getTableName(), info.getColumnName());
         }
     }),
-    ORDER_BY_ARRAY_NOT_SUPPORTED(515, "42893", "ORDER BY of an array type is not allowed"),
-    NON_EQUALITY_ARRAY_COMPARISON(516, "42894", "Array types may only be compared using = or !="),
-    INVALID_NOT_NULL_CONSTRAINT(517, "42895", "Invalid not null constraint on non primary key column"),
+    ORDER_BY_ARRAY_NOT_SUPPORTED(515, "42893", "ORDER BY of an array type is not allowed."),
+    NON_EQUALITY_ARRAY_COMPARISON(516, "42894", "Array types may only be compared using = or !=."),
+    INVALID_NOT_NULL_CONSTRAINT(517, "42895", "Invalid not null constraint on non primary key column."),
 
     /**
      *  Invalid Transaction State (errorcode 05, sqlstate 25)
      */
      READ_ONLY_CONNECTION(518,"25502","Mutations are not permitted for a read-only connection."),
 
-     VARBINARY_ARRAY_NOT_SUPPORTED(519, "42896", "VARBINARY ARRAY is not supported"),
+     VARBINARY_ARRAY_NOT_SUPPORTED(519, "42896", "VARBINARY ARRAY is not supported."),
 
      /**
       *  Expression Index exceptions.
       */
-     AGGREGATE_EXPRESSION_NOT_ALLOWED_IN_INDEX(520, "42897", "Aggregate expression not allowed in an index"),
-     NON_DETERMINISTIC_EXPRESSION_NOT_ALLOWED_IN_INDEX(521, "42898", "Non-deterministic expression not allowed in an index"),
-     STATELESS_EXPRESSION_NOT_ALLOWED_IN_INDEX(522, "42899", "Stateless expression not allowed in an index"),
+     AGGREGATE_EXPRESSION_NOT_ALLOWED_IN_INDEX(520, "42897", "Aggregate expression not allowed in an index."),
+     NON_DETERMINISTIC_EXPRESSION_NOT_ALLOWED_IN_INDEX(521, "42898", "Non-deterministic expression not allowed in an index."),
+     STATELESS_EXPRESSION_NOT_ALLOWED_IN_INDEX(522, "42899", "Stateless expression not allowed in an index."),
      
      /**
       *  Transaction exceptions.
       */
-     TRANSACTION_CONFLICT_EXCEPTION(523, "42900", "Transaction aborted due to conflict with other mutations"),
-     TRANSACTION_EXCEPTION(524, "42901", "Transaction aborted due to error"),
+     TRANSACTION_CONFLICT_EXCEPTION(523, "42900", "Transaction aborted due to conflict with other mutations."),
+     TRANSACTION_EXCEPTION(524, "42901", "Transaction aborted due to error."),
 
      /**
       * Union All related errors
       */
-     SELECT_COLUMN_NUM_IN_UNIONALL_DIFFS(525, "42902", "SELECT column number differs in a Union All query is not allowed"),
-     SELECT_COLUMN_TYPE_IN_UNIONALL_DIFFS(526, "42903", "SELECT column types differ in a Union All query is not allowed"),
+     SELECT_COLUMN_NUM_IN_UNIONALL_DIFFS(525, "42902", "SELECT column number differs in a Union All query is not allowed."),
+     SELECT_COLUMN_TYPE_IN_UNIONALL_DIFFS(526, "42903", "SELECT column types differ in a Union All query is not allowed."),
 
      /**
       * Row timestamp column related errors
       */
-     ROWTIMESTAMP_ONE_PK_COL_ONLY(527, "42904", "Only one column that is part of the primary key can be declared as a ROW_TIMESTAMP"),
-     ROWTIMESTAMP_PK_COL_ONLY(528, "42905", "Only columns part of the primary key can be declared as a ROW_TIMESTAMP"),
-     ROWTIMESTAMP_CREATE_ONLY(529, "42906", "A column can be added as ROW_TIMESTAMP only in CREATE TABLE"),
-     ROWTIMESTAMP_COL_INVALID_TYPE(530, "42907", "A column can be added as ROW_TIMESTAMP only if it is of type DATE, BIGINT, TIME OR TIMESTAMP"),
-     ROWTIMESTAMP_NOT_ALLOWED_ON_VIEW(531, "42908", "Declaring a column as row_timestamp is not allowed for views"),
-     INVALID_SCN(532, "42909", "Value of SCN cannot be less than zero"),
+     ROWTIMESTAMP_ONE_PK_COL_ONLY(527, "42904", "Only one column that is part of the primary key can be declared as a ROW_TIMESTAMP."),
+     ROWTIMESTAMP_PK_COL_ONLY(528, "42905", "Only columns part of the primary key can be declared as a ROW_TIMESTAMP."),
+     ROWTIMESTAMP_CREATE_ONLY(529, "42906", "A column can be added as ROW_TIMESTAMP only in CREATE TABLE."),
+     ROWTIMESTAMP_COL_INVALID_TYPE(530, "42907", "A column can be added as ROW_TIMESTAMP only if it is of type DATE, BIGINT, TIME OR TIMESTAMP."),
+     ROWTIMESTAMP_NOT_ALLOWED_ON_VIEW(531, "42908", "Declaring a column as row_timestamp is not allowed for views."),
+     INVALID_SCN(532, "42909", "Value of SCN cannot be less than zero."),
      /**
      * HBase and Phoenix specific implementation defined sub-classes.
      * Column family related exceptions.
@@ -241,10 +241,10 @@ public enum SQLExceptionCode {
     INVALID_INDEX_STATE_TRANSITION(1028, "42Y87", "Invalid index state transition."),
     INVALID_MUTABLE_INDEX_CONFIG(1029, "42Y88", "Mutable secondary indexes must have the "
             + IndexManagementUtil.WAL_EDIT_CODEC_CLASS_KEY + " property set to "
-            +  IndexManagementUtil.INDEX_WAL_EDIT_CODEC_CLASS_NAME + " in the hbase-sites.xml of every region server"),
+            +  IndexManagementUtil.INDEX_WAL_EDIT_CODEC_CLASS_NAME + " in the hbase-sites.xml of every region server."),
 
 
-    CANNOT_CREATE_TENANT_SPECIFIC_TABLE(1030, "42Y89", "Cannot create table for tenant-specific connection"),
+    CANNOT_CREATE_TENANT_SPECIFIC_TABLE(1030, "42Y89", "Cannot create table for tenant-specific connection."),
     DEFAULT_COLUMN_FAMILY_ONLY_ON_CREATE_TABLE(1034, "42Y93", "Default column family may only be specified when creating a table."),
     INSUFFICIENT_MULTI_TENANT_COLUMNS(1040, "42Y96", "A MULTI_TENANT table must have two or more PK columns with the first column being NOT NULL."),
     TENANTID_IS_OF_WRONG_TYPE(1041, "42Y97", "The TenantId could not be converted to correct format for this table."),
@@ -254,33 +254,33 @@ public enum SQLExceptionCode {
     NO_LOCAL_INDEX_ON_TABLE_WITH_IMMUTABLE_ROWS(1048,"43A05","Local indexes aren't allowed on tables with immutable rows."),
     COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY(1049, "43A06", "Column family not allowed for table properties."),
     COLUMN_FAMILY_NOT_ALLOWED_FOR_TTL(1050, "43A07", "Setting TTL for a column family not supported. You can only have TTL for the entire table."),
-    CANNOT_ALTER_PROPERTY(1051, "43A08", "Property can be specified or changed only when creating a table"),
-    CANNOT_SET_PROPERTY_FOR_COLUMN_NOT_ADDED(1052, "43A09", "Property cannot be specified for a column family that is not being added or modified"),
-    CANNOT_SET_TABLE_PROPERTY_ADD_COLUMN(1053, "43A10", "Table level property cannot be set when adding a column"),
+    CANNOT_ALTER_PROPERTY(1051, "43A08", "Property can be specified or changed only when creating a table."),
+    CANNOT_SET_PROPERTY_FOR_COLUMN_NOT_ADDED(1052, "43A09", "Property cannot be specified for a column family that is not being added or modified."),
+    CANNOT_SET_TABLE_PROPERTY_ADD_COLUMN(1053, "43A10", "Table level property cannot be set when adding a column."),
 
     NO_LOCAL_INDEXES(1054, "43A11", "Local secondary indexes are not supported for HBase versions " +
         MetaDataUtil.decodeHBaseVersionAsString(PhoenixDatabaseMetaData.MIN_LOCAL_SI_VERSION_DISALLOW) + " through " + MetaDataUtil.decodeHBaseVersionAsString(PhoenixDatabaseMetaData.MAX_LOCAL_SI_VERSION_DISALLOW) + " inclusive."),
     UNALLOWED_LOCAL_INDEXES(1055, "43A12", "Local secondary indexes are configured to not be allowed."),
     
-    DESC_VARBINARY_NOT_SUPPORTED(1056, "43A13", "Descending VARBINARY columns not supported"),
+    DESC_VARBINARY_NOT_SUPPORTED(1056, "43A13", "Descending VARBINARY columns not supported."),
     
-    DEFAULT_COLUMN_FAMILY_ON_SHARED_TABLE(1069, "43A13", "Default column family not allowed on VIEW or shared INDEX"),
-    ONLY_TABLE_MAY_BE_DECLARED_TRANSACTIONAL(1070, "44A01", "Only tables may be declared as transactional"),
-    TX_MAY_NOT_SWITCH_TO_NON_TX(1071, "44A02", "A transactional table may not be switched to non transactional"),
-	STORE_NULLS_MUST_BE_TRUE_FOR_TRANSACTIONAL(1072, "44A03", "Store nulls must be true when a table is transactional"),
-    CANNOT_START_TRANSACTION_WITH_SCN_SET(1073, "44A04", "Cannot start a transaction on a connection with SCN set"),
-    TX_MAX_VERSIONS_MUST_BE_GREATER_THAN_ONE(1074, "44A05", "A transactional table must define VERSION of greater than one"),
-    CANNOT_SPECIFY_SCN_FOR_TXN_TABLE(1075, "44A06", "Cannot use a connection with SCN set for a transactional table"),
-    NULL_TRANSACTION_CONTEXT(1076, "44A07", "No Transaction Context available"),
+    DEFAULT_COLUMN_FAMILY_ON_SHARED_TABLE(1069, "43A13", "Default column family not allowed on VIEW or shared INDEX."),
+    ONLY_TABLE_MAY_BE_DECLARED_TRANSACTIONAL(1070, "44A01", "Only tables may be declared as transactional."),
+    TX_MAY_NOT_SWITCH_TO_NON_TX(1071, "44A02", "A transactional table may not be switched to non transactional."),
+	STORE_NULLS_MUST_BE_TRUE_FOR_TRANSACTIONAL(1072, "44A03", "Store nulls must be true when a table is transactional."),
+    CANNOT_START_TRANSACTION_WITH_SCN_SET(1073, "44A04", "Cannot start a transaction on a connection with SCN set."),
+    TX_MAX_VERSIONS_MUST_BE_GREATER_THAN_ONE(1074, "44A05", "A transactional table must define VERSION of greater than one."),
+    CANNOT_SPECIFY_SCN_FOR_TXN_TABLE(1075, "44A06", "Cannot use a connection with SCN set for a transactional table."),
+    NULL_TRANSACTION_CONTEXT(1076, "44A07", "No Transaction Context available."),
     TRANSACTION_FAILED(1077, "44A08", "Transaction Failure "),
-    CANNOT_CREATE_TXN_TABLE_IF_TXNS_DISABLED(1078, "44A09", "Cannot create a transactional table if transactions are disabled"),
-    CANNOT_ALTER_TO_BE_TXN_IF_TXNS_DISABLED(1079, "44A10", "Cannot alter table to be transactional table if transactions are disabled"),
-    CANNOT_CREATE_TXN_TABLE_WITH_ROW_TIMESTAMP(1080, "44A11", "Cannot create a transactional table if transactions are disabled"),
-    CANNOT_ALTER_TO_BE_TXN_WITH_ROW_TIMESTAMP(1081, "44A12", "Cannot alter table to be transactional table if transactions are disabled"),
-    TX_MUST_BE_ENABLED_TO_SET_TX_CONTEXT(1082, "44A13", "Cannot set transaction context if transactions are disabled"),
-    TX_MUST_BE_ENABLED_TO_SET_AUTO_FLUSH(1083, "44A14", "Cannot set auto flush if transactions are disabled"),
-    TX_MUST_BE_ENABLED_TO_SET_ISOLATION_LEVEL(1084, "44A15", "Cannot set isolation level to TRANSACTION_READ_COMMITTED or TRANSACTION_SERIALIZABLE if transactions are disabled"),
-    TX_UNABLE_TO_GET_WRITE_FENCE(1085, "44A16", "Unable to obtain write fence for DDL operation"),
+    CANNOT_CREATE_TXN_TABLE_IF_TXNS_DISABLED(1078, "44A09", "Cannot create a transactional table if transactions are disabled."),
+    CANNOT_ALTER_TO_BE_TXN_IF_TXNS_DISABLED(1079, "44A10", "Cannot alter table to be transactional table if transactions are disabled."),
+    CANNOT_CREATE_TXN_TABLE_WITH_ROW_TIMESTAMP(1080, "44A11", "Cannot create a transactional table if transactions are disabled."),
+    CANNOT_ALTER_TO_BE_TXN_WITH_ROW_TIMESTAMP(1081, "44A12", "Cannot alter table to be transactional table if transactions are disabled."),
+    TX_MUST_BE_ENABLED_TO_SET_TX_CONTEXT(1082, "44A13", "Cannot set transaction context if transactions are disabled."),
+    TX_MUST_BE_ENABLED_TO_SET_AUTO_FLUSH(1083, "44A14", "Cannot set auto flush if transactions are disabled."),
+    TX_MUST_BE_ENABLED_TO_SET_ISOLATION_LEVEL(1084, "44A15", "Cannot set isolation level to TRANSACTION_READ_COMMITTED or TRANSACTION_SERIALIZABLE if transactions are disabled."),
+    TX_UNABLE_TO_GET_WRITE_FENCE(1085, "44A16", "Unable to obtain write fence for DDL operation."),
 
     /** Sequence related */
     SEQUENCE_ALREADY_EXIST(1200, "42Z00", "Sequence already exists.", new Factory() {
@@ -300,17 +300,17 @@ public enum SQLExceptionCode {
     CACHE_MUST_BE_NON_NEGATIVE_CONSTANT(1204, "42Z04", "Sequence CACHE value must be a non negative integer constant."),
     INVALID_USE_OF_NEXT_VALUE_FOR(1205, "42Z05", "NEXT VALUE FOR may only be used as in a SELECT or an UPSERT VALUES expression."),
     CANNOT_CALL_CURRENT_BEFORE_NEXT_VALUE(1206, "42Z06", "NEXT VALUE FOR must be called before CURRENT VALUE FOR is called."),
-    EMPTY_SEQUENCE_CACHE(1207, "42Z07", "No more cached sequence values"),
+    EMPTY_SEQUENCE_CACHE(1207, "42Z07", "No more cached sequence values."),
     MINVALUE_MUST_BE_CONSTANT(1208, "42Z08", "Sequence MINVALUE must be an integer or long constant."),
     MAXVALUE_MUST_BE_CONSTANT(1209, "42Z09", "Sequence MAXVALUE must be an integer or long constant."),
     MINVALUE_MUST_BE_LESS_THAN_OR_EQUAL_TO_MAXVALUE(1210, "42Z10", "Sequence MINVALUE must be less than or equal to MAXVALUE."),
     STARTS_WITH_MUST_BE_BETWEEN_MIN_MAX_VALUE(1211, "42Z11",
-            "STARTS WITH value must be greater than or equal to MINVALUE and less than or equal to MAXVALUE"),
-    SEQUENCE_VAL_REACHED_MAX_VALUE(1212, "42Z12", "Reached MAXVALUE of sequence"),
-    SEQUENCE_VAL_REACHED_MIN_VALUE(1213, "42Z13", "Reached MINVALUE of sequence"),
-    INCREMENT_BY_MUST_NOT_BE_ZERO(1214, "42Z14", "Sequence INCREMENT BY value cannot be zero"),
+            "STARTS WITH value must be greater than or equal to MINVALUE and less than or equal to MAXVALUE."),
+    SEQUENCE_VAL_REACHED_MAX_VALUE(1212, "42Z12", "Reached MAXVALUE of sequence."),
+    SEQUENCE_VAL_REACHED_MIN_VALUE(1213, "42Z13", "Reached MINVALUE of sequence."),
+    INCREMENT_BY_MUST_NOT_BE_ZERO(1214, "42Z14", "Sequence INCREMENT BY value cannot be zero."),
     NUM_SEQ_TO_ALLOCATE_MUST_BE_CONSTANT(1215, "42Z15", "Sequence NEXT n VALUES FOR must be a positive integer or constant." ),
-    NUM_SEQ_TO_ALLOCATE_NOT_SUPPORTED(1216, "42Z16", "Sequence NEXT n VALUES FOR is not supported for Sequences with the CYCLE flag" ),
+    NUM_SEQ_TO_ALLOCATE_NOT_SUPPORTED(1216, "42Z16", "Sequence NEXT n VALUES FOR is not supported for Sequences with the CYCLE flag." ),
 
     /** Parser error. (errorcode 06, sqlState 42P) */
     PARSER_ERROR(601, "42P00", "Syntax error.", Factory.SYNTAX_ERROR),
@@ -323,7 +323,7 @@ public enum SQLExceptionCode {
      * Implementation defined class. Execution exceptions (errorcode 11, sqlstate XCL).
      */
     RESULTSET_CLOSED(1101, "XCL01", "ResultSet is closed."),
-    GET_TABLE_REGIONS_FAIL(1102, "XCL02", "Cannot get all table regions"),
+    GET_TABLE_REGIONS_FAIL(1102, "XCL02", "Cannot get all table regions."),
     EXECUTE_QUERY_NOT_APPLICABLE(1103, "XCL03", "executeQuery may not be used."),
     EXECUTE_UPDATE_NOT_APPLICABLE(1104, "XCL04", "executeUpdate may not be used."),
     SPLIT_POINT_NOT_CONSTANT(1105, "XCL05", "Split points must be constants."),
@@ -335,22 +335,22 @@ public enum SQLExceptionCode {
             return new StaleRegionBoundaryCacheException(info.getSchemaName(), info.getTableName());
         }
     }),
-    CANNOT_SPLIT_LOCAL_INDEX(1109,"XCL09", "Local index may not be pre-split"),
-    CANNOT_SALT_LOCAL_INDEX(1110,"XCL10", "Local index may not be salted"),
+    CANNOT_SPLIT_LOCAL_INDEX(1109,"XCL09", "Local index may not be pre-split."),
+    CANNOT_SALT_LOCAL_INDEX(1110,"XCL10", "Local index may not be salted."),
 
     /**
      * Implementation defined class. Phoenix internal error. (errorcode 20, sqlstate INT).
      */
     CANNOT_CALL_METHOD_ON_TYPE(2001, "INT01", "Cannot call method on the argument type."),
-    CLASS_NOT_UNWRAPPABLE(2002, "INT03", "Class not unwrappable"),
+    CLASS_NOT_UNWRAPPABLE(2002, "INT03", "Class not unwrappable."),
     PARAM_INDEX_OUT_OF_BOUND(2003, "INT04", "Parameter position is out of range."),
-    PARAM_VALUE_UNBOUND(2004, "INT05", "Parameter value unbound"),
+    PARAM_VALUE_UNBOUND(2004, "INT05", "Parameter value unbound."),
     INTERRUPTED_EXCEPTION(2005, "INT07", "Interrupted exception."),
     INCOMPATIBLE_CLIENT_SERVER_JAR(2006, "INT08", "Incompatible jars detected between client and server."),
     OUTDATED_JARS(2007, "INT09", "Outdated jars."),
     INDEX_METADATA_NOT_FOUND(2008, "INT10", "Unable to find cached index metadata. "),
-    UNKNOWN_ERROR_CODE(2009, "INT11", "Unknown error code"),
-    OPERATION_TIMED_OUT(6000, "TIM01", "Operation timed out", new Factory() {
+    UNKNOWN_ERROR_CODE(2009, "INT11", "Unknown error code."),
+    OPERATION_TIMED_OUT(6000, "TIM01", "Operation timed out.", new Factory() {
         @Override
         public SQLException newException(SQLExceptionInfo info) {
             return new SQLTimeoutException(OPERATION_TIMED_OUT.getMessage(),

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f572fa63/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
index d3b36ec..3dfae46 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
@@ -210,12 +210,12 @@ public class MutationState implements SQLCloseable {
      * when a data table transaction is started before the create index
      * but completes after it. In this case, we need to rerun the data
      * table transaction after the index creation so that the index rows
-     * are generated. See {@link #addReadFence(PTable)} and TEPHRA-157
+     * are generated. See {@link #addDMLFence(PTable)} and TEPHRA-157
      * for more information.
      * @param dataTable the data table upon which an index is being added
      * @throws SQLException
      */
-    public void commitWriteFence(PTable dataTable) throws SQLException {
+    public void commitDDLFence(PTable dataTable) throws SQLException {
         if (dataTable.isTransactional()) {
             byte[] key = SchemaUtil.getTableKey(dataTable);
             boolean success = false;
@@ -249,12 +249,12 @@ public class MutationState implements SQLCloseable {
     /**
      * Add an entry to the change set representing the DML operation that is starting.
      * These entries will not conflict with each other, but they will conflict with a
-     * DDL operation of creating an index. See {@link #addReadFence(PTable)} and TEPHRA-157
+     * DDL operation of creating an index. See {@link #addDMLFence(PTable)} and TEPHRA-157
      * for more information.
      * @param dataTable the table which is doing DML
      * @throws SQLException
      */
-    public void addReadFence(PTable dataTable) throws SQLException {
+    public void addDMLFence(PTable dataTable) throws SQLException {
         if (this.txContext == null) {
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.NULL_TRANSACTION_CONTEXT).build().buildException();
         }
@@ -1004,7 +1004,9 @@ public class MutationState implements SQLCloseable {
                     // Keep all mutations we've encountered until a commit or rollback.
                     // This is not ideal, but there's not good way to get the values back
                     // in the event that we need to replay the commit.
-                    joinMutationState(tableRef, valuesMap, txMutations);
+                    // Copy TableRef so we have the original PTable and know when the
+                    // indexes have changed.
+                    joinMutationState(new TableRef(tableRef), valuesMap, txMutations);
                 }
                 // Remove batches as we process them
                 if (sendAll) {
@@ -1180,11 +1182,11 @@ public class MutationState implements SQLCloseable {
                             Set<TableRef> txTableRefs = txMutations.keySet();
                             for (TableRef tableRef : txTableRefs) {
                                 PTable dataTable = tableRef.getTable();
-                                addReadFence(dataTable);
+                                addDMLFence(dataTable);
                             }
                             try {
                                 // Only retry if an index was added
-                                retryCommit = wasIndexAdded(txTableRefs);
+                                retryCommit = shouldResubmitTransaction(txTableRefs);
                             } catch (SQLException e) {
                                 retryCommit = false;
                                 if (sqlE == null) {
@@ -1214,11 +1216,12 @@ public class MutationState implements SQLCloseable {
      * @return true if indexes were added and false otherwise.
      * @throws SQLException 
      */
-    private boolean wasIndexAdded(Set<TableRef> txTableRefs) throws SQLException {
+    private boolean shouldResubmitTransaction(Set<TableRef> txTableRefs) throws SQLException {
         if (logger.isInfoEnabled()) logger.info("Checking for index updates as of "  + getInitialWritePointer());
         MetaDataClient client = new MetaDataClient(connection);
         PMetaData cache = connection.getMetaDataCache();
-        boolean addedIndexes = false;
+        boolean addedAnyIndexes = false;
+        boolean allImmutableTables = !txTableRefs.isEmpty();
         for (TableRef tableRef : txTableRefs) {
             PTable dataTable = tableRef.getTable();
             List<PTable> oldIndexes;
@@ -1227,20 +1230,24 @@ public class MutationState implements SQLCloseable {
             MetaDataMutationResult result = client.updateCache(dataTable.getTenantId(), dataTable.getSchemaName().getString(), dataTable.getTableName().getString());
             long timestamp = TransactionUtil.getResolvedTime(connection, result);
             tableRef.setTimeStamp(timestamp);
-            if (result.getTable() == null) {
+            PTable updatedDataTable = result.getTable();
+            if (updatedDataTable == null) {
                 throw new TableNotFoundException(dataTable.getSchemaName().getString(), dataTable.getTableName().getString());
             }
-            tableRef.setTable(result.getTable());
-            if (!addedIndexes) {
+            allImmutableTables |= updatedDataTable.isImmutableRows();
+            tableRef.setTable(updatedDataTable);
+            if (!addedAnyIndexes) {
                 // TODO: in theory we should do a deep equals check here, as it's possible
                 // that an index was dropped and recreated with the same name but different
                 // indexed/covered columns.
-                addedIndexes = (!oldIndexes.equals(result.getTable().getIndexes()));
-                if (logger.isInfoEnabled()) logger.info((addedIndexes ? "Updates " : "No updates ") + "as of "  + timestamp + " to " + dataTable.getName().getString() + " with indexes " + tableRef.getTable().getIndexes());
+                addedAnyIndexes = (!oldIndexes.equals(updatedDataTable.getIndexes()));
+                if (logger.isInfoEnabled()) logger.info((addedAnyIndexes ? "Updates " : "No updates ") + "as of "  + timestamp + " to " + updatedDataTable.getName().getString() + " with indexes " + updatedDataTable.getIndexes());
             }
         }
-        if (logger.isInfoEnabled()) logger.info((addedIndexes ? "Updates " : "No updates ") + "to indexes as of "  + getInitialWritePointer());
-        return addedIndexes;
+        if (logger.isInfoEnabled()) logger.info((addedAnyIndexes ? "Updates " : "No updates ") + "to indexes as of "  + getInitialWritePointer() + " over " + (allImmutableTables ? " all immutable tables" : " some mutable tables"));
+        // If all tables are immutable, we know the conflict we got was due to our DDL/DML fence.
+        // If any indexes were added, then the conflict might be due to DDL/DML fence.
+        return allImmutableTables || addedAnyIndexes;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f572fa63/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
index 6bb5722..b54ccd5 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
@@ -331,7 +331,7 @@ public class PhoenixStatement implements Statement, SQLCloseable {
                                 MutationPlan plan = stmt.compilePlan(PhoenixStatement.this, Sequence.ValueOp.VALIDATE_SEQUENCE);
                                 if (plan.getTargetRef() != null && plan.getTargetRef().getTable() != null && plan.getTargetRef().getTable().isTransactional()) {
                                     state.startTransaction();
-                                    state.addReadFence(plan.getTargetRef().getTable());
+                                    state.addDMLFence(plan.getTargetRef().getTable());
                                 }
                                 Iterator<TableRef> tableRefs = plan.getSourceRefs().iterator();
                                 state.sendUncommitted(tableRefs);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f572fa63/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 d134f08..0b446b3 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
@@ -1074,7 +1074,7 @@ public class MetaDataClient {
 
                         @Override
                         public MutationState execute() throws SQLException {
-                            connection.getMutationState().commitWriteFence(dataTable);
+                            connection.getMutationState().commitDDLFence(dataTable);
                             Cell kv = plan.iterator().next().getValue(0);
                             ImmutableBytesWritable tmpPtr = new ImmutableBytesWritable(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength());
                             // A single Cell will be returned with the count(*) - we decode that here

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f572fa63/phoenix-core/src/main/java/org/apache/phoenix/schema/TableRef.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/TableRef.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/TableRef.java
index 35e2f77..8f6e271 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/TableRef.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/TableRef.java
@@ -36,12 +36,16 @@ public class TableRef {
     private final long lowerBoundTimeStamp;
     private final boolean hasDynamicCols;
 
+    public TableRef(TableRef tableRef) {
+        this(tableRef.alias, tableRef.table, tableRef.upperBoundTimeStamp, tableRef.lowerBoundTimeStamp, tableRef.hasDynamicCols);
+    }
+    
     public TableRef(TableRef tableRef, long timeStamp) {
-        this(tableRef.alias, tableRef.table, timeStamp, tableRef.hasDynamicCols);
+        this(tableRef.alias, tableRef.table, timeStamp, tableRef.lowerBoundTimeStamp, tableRef.hasDynamicCols);
     }
     
     public TableRef(TableRef tableRef, String alias) {
-        this(alias, tableRef.table, tableRef.upperBoundTimeStamp, tableRef.hasDynamicCols);
+        this(alias, tableRef.table, tableRef.upperBoundTimeStamp, tableRef.lowerBoundTimeStamp, tableRef.hasDynamicCols);
     }
     
     public TableRef(PTable table) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f572fa63/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
index 9411549..6d6dcdf 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
@@ -217,7 +217,7 @@ public class QueryCompilerTest extends BaseConnectionlessQueryTest {
             statement.execute();
             fail();
         } catch (SQLException e) {
-            assertTrue(e.getMessage(), e.getMessage().contains("ERROR 517 (42895): Invalid not null constraint on non primary key column columnName=FOO.PK"));
+            assertEquals(SQLExceptionCode.INVALID_NOT_NULL_CONSTRAINT.getErrorCode(), e.getErrorCode());
         } finally {
             conn.close();
         }


[05/50] [abbrv] phoenix git commit: PHOENIX-2478 Rows committed in transaction overlapping index creation are not populated

Posted by ma...@apache.org.
PHOENIX-2478 Rows committed in transaction overlapping index creation are not populated


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

Branch: refs/heads/calcite
Commit: 13699371820928cf14e0e2c5bbffe338c7aa2e93
Parents: f591da4
Author: James Taylor <jt...@salesforce.com>
Authored: Mon Jan 18 21:14:34 2016 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Mon Jan 18 21:14:34 2016 -0800

----------------------------------------------------------------------
 .../apache/phoenix/execute/MutationState.java   | 478 ++++++++++---------
 .../apache/phoenix/schema/MetaDataClient.java   |  31 +-
 2 files changed, 265 insertions(+), 244 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/13699371/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
index ee694e7..a6fe98d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
@@ -121,6 +121,7 @@ public class MutationState implements SQLCloseable {
     private static final Logger logger = LoggerFactory.getLogger(MutationState.class);
     private static final TransactionCodec CODEC = new TransactionCodec();
     private static final int[] EMPTY_STATEMENT_INDEX_ARRAY = new int[0];
+    private static final int MAX_COMMIT_RETRIES = 3;
     
     private final PhoenixConnection connection;
     private final long maxSize;
@@ -160,37 +161,37 @@ public class MutationState implements SQLCloseable {
     }
     
     private MutationState(long maxSize, PhoenixConnection connection, Transaction tx, TransactionContext txContext, long sizeOffset) {
-    	this(maxSize, connection, Maps.<TableRef, Map<ImmutableBytesPtr,RowMutationState>>newHashMapWithExpectedSize(5), tx, txContext);
+        this(maxSize, connection, Maps.<TableRef, Map<ImmutableBytesPtr,RowMutationState>>newHashMapWithExpectedSize(5), tx, txContext);
         this.sizeOffset = sizeOffset;
     }
     
-	MutationState(long maxSize, PhoenixConnection connection,
-			Map<TableRef, Map<ImmutableBytesPtr, RowMutationState>> mutations,
-			Transaction tx, TransactionContext txContext) {
-		this.maxSize = maxSize;
-		this.connection = connection;
-		this.mutations = mutations;
-		boolean isMetricsEnabled = connection.isRequestLevelMetricsEnabled();
-		this.mutationMetricQueue = isMetricsEnabled ? new MutationMetricQueue()
-				: NoOpMutationMetricsQueue.NO_OP_MUTATION_METRICS_QUEUE;
-		this.tx = tx;
-		if (tx == null) {
+    MutationState(long maxSize, PhoenixConnection connection,
+            Map<TableRef, Map<ImmutableBytesPtr, RowMutationState>> mutations,
+            Transaction tx, TransactionContext txContext) {
+        this.maxSize = maxSize;
+        this.connection = connection;
+        this.mutations = mutations;
+        boolean isMetricsEnabled = connection.isRequestLevelMetricsEnabled();
+        this.mutationMetricQueue = isMetricsEnabled ? new MutationMetricQueue()
+                : NoOpMutationMetricsQueue.NO_OP_MUTATION_METRICS_QUEUE;
+        this.tx = tx;
+        if (tx == null) {
             this.txAwares = Collections.emptyList();
-		    if (txContext == null) {
-    			TransactionSystemClient txServiceClient = this.connection
-    					.getQueryServices().getTransactionSystemClient();
-    			this.txContext = new TransactionContext(txServiceClient);
-		    } else {
-		        isExternalTxContext = true;
-		        this.txContext = txContext;
-		    }
-		} else {
-			// this code path is only used while running child scans, we can't pass the txContext to child scans
-			// as it is not thread safe, so we use the tx member variable
-			this.txAwares = Lists.newArrayList();
-			this.txContext = null;
-		}
-	}
+            if (txContext == null) {
+                TransactionSystemClient txServiceClient = this.connection
+                        .getQueryServices().getTransactionSystemClient();
+                this.txContext = new TransactionContext(txServiceClient);
+            } else {
+                isExternalTxContext = true;
+                this.txContext = txContext;
+            }
+        } else {
+            // this code path is only used while running child scans, we can't pass the txContext to child scans
+            // as it is not thread safe, so we use the tx member variable
+            this.txAwares = Lists.newArrayList();
+            this.txContext = null;
+        }
+    }
 
     public MutationState(TableRef table, Map<ImmutableBytesPtr,RowMutationState> mutations, long sizeOffset, long maxSize, PhoenixConnection connection) {
         this(maxSize, connection, null, null, sizeOffset);
@@ -217,9 +218,11 @@ public class MutationState implements SQLCloseable {
     public void commitWriteFence(PTable dataTable) throws SQLException {
         if (dataTable.isTransactional()) {
             byte[] key = SchemaUtil.getTableKey(dataTable);
+            boolean success = false;
             try {
                 FenceWait fenceWait = VisibilityFence.prepareWait(key, connection.getQueryServices().getTransactionSystemClient());
                 fenceWait.await(10000, TimeUnit.MILLISECONDS);
+                success = true;
             } catch (InterruptedException e) {
                 Thread.currentThread().interrupt();
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.INTERRUPTED_EXCEPTION).setRootCause(e).build().buildException();
@@ -235,6 +238,7 @@ public class MutationState implements SQLCloseable {
                 // TODO: seems like an autonomous tx capability in Tephra would be useful here.
                 try {
                     txContext.start();
+                    if (logger.isInfoEnabled() && success) logger.info("Added write fence at ~" + getTransaction().getReadPointer());
                 } catch (TransactionFailureException e) {
                     throw TransactionUtil.getTransactionFailureException(e);
                 }
@@ -306,18 +310,18 @@ public class MutationState implements SQLCloseable {
             }
             if (hasUncommittedData) {
                 try {
-                	if (txContext == null) {
-                		currentTx = tx = connection.getQueryServices().getTransactionSystemClient().checkpoint(currentTx);
-                	}  else {
-                		txContext.checkpoint();
-                		currentTx = tx = txContext.getCurrentTransaction();
-                	}
+                    if (txContext == null) {
+                        currentTx = tx = connection.getQueryServices().getTransactionSystemClient().checkpoint(currentTx);
+                    }  else {
+                        txContext.checkpoint();
+                        currentTx = tx = txContext.getCurrentTransaction();
+                    }
                     // Since we've checkpointed, we can clear out uncommitted set, since a statement run afterwards
                     // should see all this data.
                     uncommittedPhysicalNames.clear();
                 } catch (TransactionFailureException e) {
                     throw new SQLException(e);
-				} 
+                } 
             }
             // Since we're querying our own table while mutating it, we must exclude
             // see our current mutations, otherwise we can get erroneous results (for DELETE)
@@ -356,7 +360,7 @@ public class MutationState implements SQLCloseable {
     }
     
     public PhoenixConnection getConnection() {
-    	return connection;
+        return connection;
     }
     
     // Kept private as the Transaction may change when check pointed. Keeping it private ensures
@@ -366,7 +370,7 @@ public class MutationState implements SQLCloseable {
     }
     
     public boolean isTransactionStarted() {
-    	return getTransaction() != null;
+        return getTransaction() != null;
     }
     
     public long getInitialWritePointer() {
@@ -391,11 +395,11 @@ public class MutationState implements SQLCloseable {
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.NULL_TRANSACTION_CONTEXT).build().buildException();
         }
         
-		if (connection.getSCN() != null) {
-			throw new SQLExceptionInfo.Builder(
-					SQLExceptionCode.CANNOT_START_TRANSACTION_WITH_SCN_SET)
-					.build().buildException();
-		}
+        if (connection.getSCN() != null) {
+            throw new SQLExceptionInfo.Builder(
+                    SQLExceptionCode.CANNOT_START_TRANSACTION_WITH_SCN_SET)
+                    .build().buildException();
+        }
         
         try {
             if (!isTransactionStarted()) {
@@ -460,9 +464,9 @@ public class MutationState implements SQLCloseable {
                 // Loop through new rows and replace existing with new
                 for (Map.Entry<ImmutableBytesPtr,RowMutationState> rowEntry : entry.getValue().entrySet()) {
                     // Replace existing row with new row
-                	RowMutationState existingRowMutationState = existingRows.put(rowEntry.getKey(), rowEntry.getValue());
+                    RowMutationState existingRowMutationState = existingRows.put(rowEntry.getKey(), rowEntry.getValue());
                     if (existingRowMutationState != null) {
-                    	Map<PColumn,byte[]> existingValues = existingRowMutationState.getColumnValues();
+                        Map<PColumn,byte[]> existingValues = existingRowMutationState.getColumnValues();
                         if (existingValues != PRow.DELETE_MARKER) {
                             Map<PColumn,byte[]> newRow = rowEntry.getValue().getColumnValues();
                             // if new row is PRow.DELETE_MARKER, it means delete, and we don't need to merge it with existing row. 
@@ -502,7 +506,7 @@ public class MutationState implements SQLCloseable {
     }
     
 
-	private static ImmutableBytesPtr getNewRowKeyWithRowTimestamp(ImmutableBytesPtr ptr, long rowTimestamp, PTable table) {
+    private static ImmutableBytesPtr getNewRowKeyWithRowTimestamp(ImmutableBytesPtr ptr, long rowTimestamp, PTable table) {
         RowKeySchema schema = table.getRowKeySchema();
         int rowTimestampColPos = table.getRowTimestampColPos();
         Field rowTimestampField = schema.getField(rowTimestampColPos); 
@@ -523,7 +527,7 @@ public class MutationState implements SQLCloseable {
         return ptr;
     }
     
-	private Iterator<Pair<byte[],List<Mutation>>> addRowMutations(final TableRef tableRef, final Map<ImmutableBytesPtr, RowMutationState> values, final long timestamp, boolean includeMutableIndexes, final boolean sendAll) { 
+    private Iterator<Pair<byte[],List<Mutation>>> addRowMutations(final TableRef tableRef, final Map<ImmutableBytesPtr, RowMutationState> values, final long timestamp, boolean includeMutableIndexes, final boolean sendAll) { 
         final PTable table = tableRef.getTable();
         final Iterator<PTable> indexes = // Only maintain tables with immutable rows through this client-side mechanism
                 (table.isImmutableRows() || includeMutableIndexes) ? 
@@ -554,13 +558,13 @@ public class MutationState implements SQLCloseable {
                                 connection.getKeyValueBuilder(), connection);
                     // we may also have to include delete mutations for immutable tables if we are not processing all the tables in the mutations map
                     if (!sendAll) {
-	                    TableRef key = new TableRef(index);
-						Map<ImmutableBytesPtr, RowMutationState> rowToColumnMap = mutations.remove(key);
-	                    if (rowToColumnMap!=null) {
-		                    final List<Mutation> deleteMutations = Lists.newArrayList();
-		                    generateMutations(tableRef, timestamp, rowToColumnMap, deleteMutations, null);
-		                    indexMutations.addAll(deleteMutations);
-	                    }
+                        TableRef key = new TableRef(index);
+                        Map<ImmutableBytesPtr, RowMutationState> rowToColumnMap = mutations.remove(key);
+                        if (rowToColumnMap!=null) {
+                            final List<Mutation> deleteMutations = Lists.newArrayList();
+                            generateMutations(tableRef, timestamp, rowToColumnMap, deleteMutations, null);
+                            indexMutations.addAll(deleteMutations);
+                        }
                     }
                 } catch (SQLException e) {
                     throw new IllegalDataException(e);
@@ -676,32 +680,32 @@ public class MutationState implements SQLCloseable {
         };
     }
         
-	/**
-	 * Validates that the meta data is valid against the server meta data if we haven't yet done so.
-	 * Otherwise, for every UPSERT VALUES call, we'd need to hit the server to see if the meta data
-	 * has changed.
-	 * @param connection
-	 * @return the server time to use for the upsert
-	 * @throws SQLException if the table or any columns no longer exist
-	 */
-	private long[] validateAll() throws SQLException {
-	    int i = 0;
-	    long[] timeStamps = new long[this.mutations.size()];
-	    for (Map.Entry<TableRef, Map<ImmutableBytesPtr,RowMutationState>> entry : mutations.entrySet()) {
-	        TableRef tableRef = entry.getKey();
-	        timeStamps[i++] = validate(tableRef, entry.getValue());
-	    }
-	    return timeStamps;
-	}
-	
-	private long validate(TableRef tableRef, Map<ImmutableBytesPtr, RowMutationState> rowKeyToColumnMap) throws SQLException {
-	    Long scn = connection.getSCN();
-	    MetaDataClient client = new MetaDataClient(connection);
-	    long serverTimeStamp = tableRef.getTimeStamp();
-	    // If we're auto committing, we've already validated the schema when we got the ColumnResolver,
-	    // so no need to do it again here.
-	    if (!connection.getAutoCommit()) {
-	        PTable table = tableRef.getTable();
+    /**
+     * Validates that the meta data is valid against the server meta data if we haven't yet done so.
+     * Otherwise, for every UPSERT VALUES call, we'd need to hit the server to see if the meta data
+     * has changed.
+     * @param connection
+     * @return the server time to use for the upsert
+     * @throws SQLException if the table or any columns no longer exist
+     */
+    private long[] validateAll() throws SQLException {
+        int i = 0;
+        long[] timeStamps = new long[this.mutations.size()];
+        for (Map.Entry<TableRef, Map<ImmutableBytesPtr,RowMutationState>> entry : mutations.entrySet()) {
+            TableRef tableRef = entry.getKey();
+            timeStamps[i++] = validate(tableRef, entry.getValue());
+        }
+        return timeStamps;
+    }
+    
+    private long validate(TableRef tableRef, Map<ImmutableBytesPtr, RowMutationState> rowKeyToColumnMap) throws SQLException {
+        Long scn = connection.getSCN();
+        MetaDataClient client = new MetaDataClient(connection);
+        long serverTimeStamp = tableRef.getTimeStamp();
+        // If we're auto committing, we've already validated the schema when we got the ColumnResolver,
+        // so no need to do it again here.
+        if (!connection.getAutoCommit()) {
+            PTable table = tableRef.getTable();
             MetaDataMutationResult result = client.updateCache(table.getSchemaName().getString(), table.getTableName().getString());
             PTable resolvedTable = result.getTable();
             if (resolvedTable == null) {
@@ -717,14 +721,14 @@ public class MutationState implements SQLCloseable {
                     // TODO: use bitset?
                     PColumn[] columns = new PColumn[resolvedTable.getColumns().size()];
                     for (Map.Entry<ImmutableBytesPtr,RowMutationState> rowEntry : rowKeyToColumnMap.entrySet()) {
-                    	RowMutationState valueEntry = rowEntry.getValue();
+                        RowMutationState valueEntry = rowEntry.getValue();
                         if (valueEntry != null) {
-                        	Map<PColumn, byte[]> colValues = valueEntry.getColumnValues();
-                        	if (colValues != PRow.DELETE_MARKER) {
+                            Map<PColumn, byte[]> colValues = valueEntry.getColumnValues();
+                            if (colValues != PRow.DELETE_MARKER) {
                                 for (PColumn column : colValues.keySet()) {
                                     columns[column.getPosition()] = column;
                                 }
-                        	}
+                            }
                         }
                     }
                     for (PColumn column : columns) {
@@ -735,8 +739,8 @@ public class MutationState implements SQLCloseable {
                 }
             }
         }
-	    return scn == null ? serverTimeStamp == QueryConstants.UNSET_TIMESTAMP ? HConstants.LATEST_TIMESTAMP : serverTimeStamp : scn;
-	}
+        return scn == null ? serverTimeStamp == QueryConstants.UNSET_TIMESTAMP ? HConstants.LATEST_TIMESTAMP : serverTimeStamp : scn;
+    }
     
     private static long calculateMutationSize(List<Mutation> mutations) {
         long byteSize = 0;
@@ -845,74 +849,74 @@ public class MutationState implements SQLCloseable {
         // add tracing for this operation
         try (TraceScope trace = Tracing.startNewSpan(connection, "Committing mutations to tables")) {
             Span span = trace.getSpan();
-	        ImmutableBytesWritable indexMetaDataPtr = new ImmutableBytesWritable();
-	        boolean isTransactional;
-	        while (tableRefIterator.hasNext()) {
-	        	// at this point we are going through mutations for each table
-	            final TableRef tableRef = tableRefIterator.next();
-	            valuesMap = mutations.get(tableRef);
-	            if (valuesMap == null || valuesMap.isEmpty()) {
-	                continue;
-	            }
+            ImmutableBytesWritable indexMetaDataPtr = new ImmutableBytesWritable();
+            boolean isTransactional;
+            while (tableRefIterator.hasNext()) {
+                // at this point we are going through mutations for each table
+                final TableRef tableRef = tableRefIterator.next();
+                valuesMap = mutations.get(tableRef);
+                if (valuesMap == null || valuesMap.isEmpty()) {
+                    continue;
+                }
                 // Validate as we go if transactional since we can undo if a problem occurs (which is unlikely)
                 long serverTimestamp = serverTimeStamps == null ? validate(tableRef, valuesMap) : serverTimeStamps[i++];
-	            final PTable table = tableRef.getTable();
-	            // Track tables to which we've sent uncommitted data
-	            if (isTransactional = table.isTransactional()) {
-	                txTableRefs.add(tableRef);
-	                uncommittedPhysicalNames.add(table.getPhysicalName().getString());
-	            }
-	            boolean isDataTable = true;
+                final PTable table = tableRef.getTable();
+                // Track tables to which we've sent uncommitted data
+                if (isTransactional = table.isTransactional()) {
+                    txTableRefs.add(tableRef);
+                    uncommittedPhysicalNames.add(table.getPhysicalName().getString());
+                }
+                boolean isDataTable = true;
                 table.getIndexMaintainers(indexMetaDataPtr, connection);
-	            Iterator<Pair<byte[],List<Mutation>>> mutationsIterator = addRowMutations(tableRef, valuesMap, serverTimestamp, false, sendAll);
-	            while (mutationsIterator.hasNext()) {
-	                Pair<byte[],List<Mutation>> pair = mutationsIterator.next();
-	                byte[] htableName = pair.getFirst();
-	                List<Mutation> mutationList = pair.getSecond();
-	                
-	                //create a span per target table
-	                //TODO maybe we can be smarter about the table name to string here?
-	                Span child = Tracing.child(span,"Writing mutation batch for table: "+Bytes.toString(htableName));
-	
-	                int retryCount = 0;
-	                boolean shouldRetry = false;
-	                do {
-	                    final ServerCache cache = isDataTable ? setMetaDataOnMutations(tableRef, mutationList, indexMetaDataPtr) : null;
-	                
-	                    // If we haven't retried yet, retry for this case only, as it's possible that
-	                    // a split will occur after we send the index metadata cache to all known
-	                    // region servers.
-	                    shouldRetry = cache != null;
-	                    SQLException sqlE = null;
-	                    HTableInterface hTable = connection.getQueryServices().getTable(htableName);
-	                    try {
-	                        if (isTransactional) {
-	                            // If we have indexes, wrap the HTable in a delegate HTable that
-	                            // will attach the necessary index meta data in the event of a
-	                            // rollback
-	                            if (!table.getIndexes().isEmpty()) {
-	                                hTable = new MetaDataAwareHTable(hTable, tableRef);
-	                            }
-	                            TransactionAwareHTable txnAware = TransactionUtil.getTransactionAwareHTable(hTable, table);
-	                            // Don't add immutable indexes (those are the only ones that would participate
-	                            // during a commit), as we don't need conflict detection for these.
-	                            if (isDataTable) {
-	                                // Even for immutable, we need to do this so that an abort has the state
-	                                // necessary to generate the rows to delete.
-	                                addTransactionParticipant(txnAware);
-	                            } else {
-	                                txnAware.startTx(getTransaction());
-	                            }
-	                            hTable = txnAware;
-	                        }
-	                        long numMutations = mutationList.size();
+                Iterator<Pair<byte[],List<Mutation>>> mutationsIterator = addRowMutations(tableRef, valuesMap, serverTimestamp, false, sendAll);
+                while (mutationsIterator.hasNext()) {
+                    Pair<byte[],List<Mutation>> pair = mutationsIterator.next();
+                    byte[] htableName = pair.getFirst();
+                    List<Mutation> mutationList = pair.getSecond();
+                    
+                    //create a span per target table
+                    //TODO maybe we can be smarter about the table name to string here?
+                    Span child = Tracing.child(span,"Writing mutation batch for table: "+Bytes.toString(htableName));
+    
+                    int retryCount = 0;
+                    boolean shouldRetry = false;
+                    do {
+                        final ServerCache cache = isDataTable ? setMetaDataOnMutations(tableRef, mutationList, indexMetaDataPtr) : null;
+                    
+                        // If we haven't retried yet, retry for this case only, as it's possible that
+                        // a split will occur after we send the index metadata cache to all known
+                        // region servers.
+                        shouldRetry = cache != null;
+                        SQLException sqlE = null;
+                        HTableInterface hTable = connection.getQueryServices().getTable(htableName);
+                        try {
+                            if (isTransactional) {
+                                // If we have indexes, wrap the HTable in a delegate HTable that
+                                // will attach the necessary index meta data in the event of a
+                                // rollback
+                                if (!table.getIndexes().isEmpty()) {
+                                    hTable = new MetaDataAwareHTable(hTable, tableRef);
+                                }
+                                TransactionAwareHTable txnAware = TransactionUtil.getTransactionAwareHTable(hTable, table);
+                                // Don't add immutable indexes (those are the only ones that would participate
+                                // during a commit), as we don't need conflict detection for these.
+                                if (isDataTable) {
+                                    // Even for immutable, we need to do this so that an abort has the state
+                                    // necessary to generate the rows to delete.
+                                    addTransactionParticipant(txnAware);
+                                } else {
+                                    txnAware.startTx(getTransaction());
+                                }
+                                hTable = txnAware;
+                            }
+                            long numMutations = mutationList.size();
                             GLOBAL_MUTATION_BATCH_SIZE.update(numMutations);
                             
                             long startTime = System.currentTimeMillis();
                             child.addTimelineAnnotation("Attempt " + retryCount);
-	                        hTable.batch(mutationList);
-	                        child.stop();
-	                        child.stop();
+                            hTable.batch(mutationList);
+                            child.stop();
+                            child.stop();
                             shouldRetry = false;
                             long mutationCommitTime = System.currentTimeMillis() - startTime;
                             GLOBAL_MUTATION_COMMIT_TIME.update(mutationCommitTime);
@@ -920,80 +924,80 @@ public class MutationState implements SQLCloseable {
                             long mutationSizeBytes = calculateMutationSize(mutationList);
                             MutationMetric mutationsMetric = new MutationMetric(numMutations, mutationSizeBytes, mutationCommitTime);
                             mutationMetricQueue.addMetricsForTable(Bytes.toString(htableName), mutationsMetric);
-	                    } catch (Exception e) {
-	                        SQLException inferredE = ServerUtil.parseServerExceptionOrNull(e);
-	                        if (inferredE != null) {
-	                            if (shouldRetry && retryCount == 0 && inferredE.getErrorCode() == SQLExceptionCode.INDEX_METADATA_NOT_FOUND.getErrorCode()) {
-	                                // Swallow this exception once, as it's possible that we split after sending the index metadata
-	                                // and one of the region servers doesn't have it. This will cause it to have it the next go around.
-	                                // If it fails again, we don't retry.
-	                                String msg = "Swallowing exception and retrying after clearing meta cache on connection. " + inferredE;
-	                                logger.warn(LogUtil.addCustomAnnotations(msg, connection));
-	                                connection.getQueryServices().clearTableRegionCache(htableName);
-	
-	                                // add a new child span as this one failed
-	                                child.addTimelineAnnotation(msg);
-	                                child.stop();
-	                                child = Tracing.child(span,"Failed batch, attempting retry");
-	
-	                                continue;
-	                            }
-	                            e = inferredE;
-	                        }
-	                        // Throw to client an exception that indicates the statements that
-	                        // were not committed successfully.
-	                        sqlE = new CommitException(e, getUncommittedStatementIndexes());
-	                    } finally {
-	                        try {
-	                            if (cache != null) {
-	                                cache.close();
-	                            }
-	                        } finally {
-	                            try {
-	                                hTable.close();
-	                            } 
-	                            catch (IOException e) {
-	                                if (sqlE != null) {
-	                                    sqlE.setNextException(ServerUtil.parseServerException(e));
-	                                } else {
-	                                    sqlE = ServerUtil.parseServerException(e);
-	                                }
-	                            } 
-	                            if (sqlE != null) {
-	                                throw sqlE;
-	                            }
-	                        }
-	                    }
-	                } while (shouldRetry && retryCount++ < 1);
-	                isDataTable = false;
-	            }
-	            if (tableRef.getTable().getType() != PTableType.INDEX) {
-	                numRows -= valuesMap.size();
-	            }
-	            // For transactions, track the statement indexes as we send data
-	            // over because our CommitException should include all statements
-	            // involved in the transaction since none of them would have been
-	            // committed in the event of a failure.
-	            if (isTransactional) {
-	                addUncommittedStatementIndexes(valuesMap.values());
-	                if (txMutations == null) {
-	                    txMutations = Maps.newHashMapWithExpectedSize(mutations.size());
-	                }
-	                // Keep all mutations we've encountered until a commit or rollback.
-	                // This is not ideal, but there's not good way to get the values back
-	                // in the event that we need to replay the commit.
-	                txMutations.put(tableRef, valuesMap);
-	            }
+                        } catch (Exception e) {
+                            SQLException inferredE = ServerUtil.parseServerExceptionOrNull(e);
+                            if (inferredE != null) {
+                                if (shouldRetry && retryCount == 0 && inferredE.getErrorCode() == SQLExceptionCode.INDEX_METADATA_NOT_FOUND.getErrorCode()) {
+                                    // Swallow this exception once, as it's possible that we split after sending the index metadata
+                                    // and one of the region servers doesn't have it. This will cause it to have it the next go around.
+                                    // If it fails again, we don't retry.
+                                    String msg = "Swallowing exception and retrying after clearing meta cache on connection. " + inferredE;
+                                    logger.warn(LogUtil.addCustomAnnotations(msg, connection));
+                                    connection.getQueryServices().clearTableRegionCache(htableName);
+    
+                                    // add a new child span as this one failed
+                                    child.addTimelineAnnotation(msg);
+                                    child.stop();
+                                    child = Tracing.child(span,"Failed batch, attempting retry");
+    
+                                    continue;
+                                }
+                                e = inferredE;
+                            }
+                            // Throw to client an exception that indicates the statements that
+                            // were not committed successfully.
+                            sqlE = new CommitException(e, getUncommittedStatementIndexes());
+                        } finally {
+                            try {
+                                if (cache != null) {
+                                    cache.close();
+                                }
+                            } finally {
+                                try {
+                                    hTable.close();
+                                } 
+                                catch (IOException e) {
+                                    if (sqlE != null) {
+                                        sqlE.setNextException(ServerUtil.parseServerException(e));
+                                    } else {
+                                        sqlE = ServerUtil.parseServerException(e);
+                                    }
+                                } 
+                                if (sqlE != null) {
+                                    throw sqlE;
+                                }
+                            }
+                        }
+                    } while (shouldRetry && retryCount++ < 1);
+                    isDataTable = false;
+                }
+                if (tableRef.getTable().getType() != PTableType.INDEX) {
+                    numRows -= valuesMap.size();
+                }
+                // For transactions, track the statement indexes as we send data
+                // over because our CommitException should include all statements
+                // involved in the transaction since none of them would have been
+                // committed in the event of a failure.
+                if (isTransactional) {
+                    addUncommittedStatementIndexes(valuesMap.values());
+                    if (txMutations == null) {
+                        txMutations = Maps.newHashMapWithExpectedSize(mutations.size());
+                    }
+                    // Keep all mutations we've encountered until a commit or rollback.
+                    // This is not ideal, but there's not good way to get the values back
+                    // in the event that we need to replay the commit.
+                    txMutations.put(tableRef, valuesMap);
+                }
                 // Remove batches as we process them
-	            if (sendAll) {
-	                // Iterating through map key set in this case, so we cannot use
-	                // the remove method without getting a concurrent modification
-	                // exception.
-	            	tableRefIterator.remove();
-	            } else {
-	            	mutations.remove(tableRef);
-	            }
-	        }
+                if (sendAll) {
+                    // Iterating through map key set in this case, so we cannot use
+                    // the remove method without getting a concurrent modification
+                    // exception.
+                    tableRefIterator.remove();
+                } else {
+                    mutations.remove(tableRef);
+                }
+            }
         }
     }
 
@@ -1006,7 +1010,7 @@ public class MutationState implements SQLCloseable {
     }
     
     public static Transaction decodeTransaction(byte[] txnBytes) throws IOException {
-    	return (txnBytes == null || txnBytes.length==0) ? null : CODEC.decode(txnBytes);
+        return (txnBytes == null || txnBytes.length==0) ? null : CODEC.decode(txnBytes);
     }
 
     private ServerCache setMetaDataOnMutations(TableRef tableRef, List<? extends Mutation> mutations,
@@ -1059,10 +1063,10 @@ public class MutationState implements SQLCloseable {
     }
     
     private int[] getUncommittedStatementIndexes() {
-    	for (Map<ImmutableBytesPtr, RowMutationState> rowMutationMap : mutations.values()) {
-    	    addUncommittedStatementIndexes(rowMutationMap.values());
-    	}
-    	return uncommittedStatementIndexes;
+        for (Map<ImmutableBytesPtr, RowMutationState> rowMutationMap : mutations.values()) {
+            addUncommittedStatementIndexes(rowMutationMap.values());
+        }
+        return uncommittedStatementIndexes;
     }
     
     @Override
@@ -1101,9 +1105,9 @@ public class MutationState implements SQLCloseable {
         Map<TableRef, Map<ImmutableBytesPtr,RowMutationState>> txMutations = Collections.emptyMap();
         int retryCount = 0;
         do {
-        	boolean sendSuccessful=false;
-        	boolean retryCommit = false;
-        	SQLException sqlE = null;
+            boolean sendSuccessful=false;
+            boolean retryCommit = false;
+            SQLException sqlE = null;
             try {
                 send();
                 txMutations = this.txMutations;
@@ -1121,7 +1125,8 @@ public class MutationState implements SQLCloseable {
                                 finishSuccessful = true;
                             }
                         } catch (TransactionFailureException e) {
-                            retryCommit = (e instanceof TransactionConflictException && retryCount == 0);
+                            if (logger.isInfoEnabled()) logger.info(e.getClass().getName() + " at timestamp " + getInitialWritePointer() + " with retry count of " + retryCount);
+                            retryCommit = (e instanceof TransactionConflictException && retryCount < MAX_COMMIT_RETRIES);
                             txFailure = e;
                             SQLException nextE = TransactionUtil.getTransactionFailureException(e);
                             if (sqlE == null) {
@@ -1134,7 +1139,9 @@ public class MutationState implements SQLCloseable {
                             if (!finishSuccessful) {
                                 try {
                                     txContext.abort(txFailure);
+                                    if (logger.isInfoEnabled()) logger.info("Abort successful");
                                 } catch (TransactionFailureException e) {
+                                    if (logger.isInfoEnabled()) logger.info("Abort failed with " + e);
                                     SQLException nextE = TransactionUtil.getTransactionFailureException(e);
                                     if (sqlE == null) {
                                         sqlE = nextE;
@@ -1151,8 +1158,15 @@ public class MutationState implements SQLCloseable {
                     } finally {
                         if (retryCommit) {
                             startTransaction();
+                            // Add back read fences
+                            Set<TableRef> txTableRefs = txMutations.keySet();
+                            for (TableRef tableRef : txTableRefs) {
+                                PTable dataTable = tableRef.getTable();
+                                addReadFence(dataTable);
+                            }
                             try {
-                                retryCommit = wasIndexAdded(txMutations.keySet());
+                                // Only retry if an index was added
+                                retryCommit = wasIndexAdded(txTableRefs);
                             } catch (SQLException e) {
                                 retryCommit = false;
                                 if (sqlE == null) {
@@ -1173,7 +1187,9 @@ public class MutationState implements SQLCloseable {
                 break;
             }
             retryCount++;
-            mutations.putAll(txMutations);
+            if (txMutations != null) {
+                mutations.putAll(txMutations);
+            }
         } while (true);
     }
 
@@ -1183,6 +1199,7 @@ public class MutationState implements SQLCloseable {
      * @throws SQLException 
      */
     private boolean wasIndexAdded(Set<TableRef> txTableRefs) throws SQLException {
+        if (logger.isInfoEnabled()) logger.info("Checking for index updates as of "  + getInitialWritePointer());
         MetaDataClient client = new MetaDataClient(connection);
         PMetaData cache = connection.getMetaDataCache();
         boolean addedIndexes = false;
@@ -1198,6 +1215,7 @@ public class MutationState implements SQLCloseable {
                 throw new TableNotFoundException(dataTable.getSchemaName().getString(), dataTable.getTableName().getString());
             }
             if (!result.wasUpdated()) {
+                if (logger.isInfoEnabled()) logger.info("No updates to " + dataTable.getName().getString() + " as of "  + timestamp);
                 continue;
             }
             if (!addedIndexes) {
@@ -1205,8 +1223,10 @@ public class MutationState implements SQLCloseable {
                 // that an index was dropped and recreated with the same name but different
                 // indexed/covered columns.
                 addedIndexes = (!oldIndexes.equals(result.getTable().getIndexes()));
+                if (logger.isInfoEnabled()) logger.info((addedIndexes ? "Updates " : "No updates ") + "as of "  + timestamp + " to " + dataTable.getName().getString() + " with indexes " + dataTable.getIndexes());
             }
         }
+        if (logger.isInfoEnabled()) logger.info((addedIndexes ? "Updates " : "No updates ") + "to indexes as of "  + getInitialWritePointer());
         return addedIndexes;
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/13699371/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 ee212ed..d134f08 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
@@ -479,8 +479,8 @@ public class MetaDataClient {
         // Do not make rpc to getTable if 
         // 1. table is a system table
         // 2. table was already resolved as of that timestamp
-		if (table != null && !alwaysHitServer
-				&& (systemTable || resolvedTimestamp == tableResolvedTimestamp || connection.getMetaDataCache().getAge(tableRef) < table.getUpdateCacheFrequency())) {
+        if (table != null && !alwaysHitServer
+                && (systemTable || resolvedTimestamp == tableResolvedTimestamp || connection.getMetaDataCache().getAge(tableRef) < table.getUpdateCacheFrequency())) {
             return new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS, QueryConstants.UNSET_TIMESTAMP, table);
         }
 
@@ -1383,6 +1383,7 @@ public class MetaDataClient {
             return new MutationState(0,connection);
         }
 
+        if (logger.isInfoEnabled()) logger.info("Created index " + table.getName().getString() + " at " + table.getTimeStamp());
         // In async process, we return immediately as the MR job needs to be triggered .
         if(statement.isAsync()) {
             return new MutationState(0, connection);
@@ -2887,19 +2888,19 @@ public class MetaDataClient {
                     String fullTableName = SchemaUtil.getTableName(schemaName, tableName);
                     long resolvedTimeStamp = TransactionUtil.getResolvedTime(connection, result);
                     if (table.getIndexes().isEmpty() || (numPkColumnsAdded==0 && !nonTxToTx)) {
-						connection.addColumn(
-								tenantId,
-								fullTableName,
-								columns,
-								result.getMutationTime(),
-								seqNum,
-								isImmutableRows == null ? table.isImmutableRows() : isImmutableRows,
-								disableWAL == null ? table.isWALDisabled() : disableWAL,
-								multiTenant == null ? table.isMultiTenant() : multiTenant,
-								storeNulls == null ? table.getStoreNulls() : storeNulls, 
-								isTransactional == null ? table.isTransactional() : isTransactional,
-								updateCacheFrequency == null ? table.getUpdateCacheFrequency() : updateCacheFrequency,
-								resolvedTimeStamp);
+                        connection.addColumn(
+                                tenantId,
+                                fullTableName,
+                                columns,
+                                result.getMutationTime(),
+                                seqNum,
+                                isImmutableRows == null ? table.isImmutableRows() : isImmutableRows,
+                                disableWAL == null ? table.isWALDisabled() : disableWAL,
+                                multiTenant == null ? table.isMultiTenant() : multiTenant,
+                                storeNulls == null ? table.getStoreNulls() : storeNulls, 
+                                isTransactional == null ? table.isTransactional() : isTransactional,
+                                updateCacheFrequency == null ? table.getUpdateCacheFrequency() : updateCacheFrequency,
+                                resolvedTimeStamp);
                     } else if (updateCacheFrequency != null) {
                         // Force removal from cache as the update cache frequency has changed
                         // Note that clients outside this JVM won't be affected.


[30/50] [abbrv] phoenix git commit: PHOENIX-2625 Add @Ignore tag for IndexLoadBalancerIT#testRoundRobinAssignmentDuringMasterStartUp

Posted by ma...@apache.org.
PHOENIX-2625 Add @Ignore tag for IndexLoadBalancerIT#testRoundRobinAssignmentDuringMasterStartUp


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

Branch: refs/heads/calcite
Commit: a6cd49253de8d198c3772bbfa414cef9c361076d
Parents: 5d61fb8
Author: Samarth <sa...@salesforce.com>
Authored: Fri Jan 22 13:15:33 2016 -0800
Committer: Samarth <sa...@salesforce.com>
Committed: Fri Jan 22 13:15:33 2016 -0800

----------------------------------------------------------------------
 .../apache/phoenix/hbase/index/balancer/IndexLoadBalancerIT.java   | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6cd4925/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/balancer/IndexLoadBalancerIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/balancer/IndexLoadBalancerIT.java b/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/balancer/IndexLoadBalancerIT.java
index 1fdc8cd..1bec821 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/balancer/IndexLoadBalancerIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/balancer/IndexLoadBalancerIT.java
@@ -54,6 +54,7 @@ import org.apache.phoenix.hbase.index.Indexer;
 import org.apache.phoenix.hbase.index.master.IndexMasterObserver;
 import org.apache.phoenix.util.ConfigUtil;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -389,6 +390,7 @@ public class IndexLoadBalancerIT {
 
     }
     
+    @Ignore // FIXME: PHOENIX-2625 
     @Test
     public void testRoundRobinAssignmentDuringMasterStartUp() throws Exception {
         MiniHBaseCluster cluster = UTIL.getHBaseCluster();


[45/50] [abbrv] phoenix git commit: Revert "PHOENIX-2543 Modify StatsCollectorIT to test transactional test cases"

Posted by ma...@apache.org.
Revert "PHOENIX-2543 Modify StatsCollectorIT to test transactional test cases"

This reverts commit 791a27cfd141af969bc3e2e25ac9c14bd1b17a90.


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

Branch: refs/heads/calcite
Commit: 8a799b8cbbf7ff7deb97a48fab215a4d1895b4a1
Parents: c89903e
Author: Thomas D'Silva <td...@salesforce.com>
Authored: Thu Jan 28 15:46:49 2016 -0800
Committer: Thomas D'Silva <td...@salesforce.com>
Committed: Thu Jan 28 15:46:49 2016 -0800

----------------------------------------------------------------------
 .../phoenix/end2end/StatsCollectorIT.java       |  73 ++++------
 .../org/apache/phoenix/tx/TransactionIT.java    | 137 ++++---------------
 2 files changed, 58 insertions(+), 152 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/8a799b8c/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java
index caba259..706ae4a 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java
@@ -30,8 +30,6 @@ import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
-import java.util.Arrays;
-import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
@@ -45,23 +43,14 @@ import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
-import org.apache.phoenix.util.SchemaUtil;
-import org.apache.phoenix.util.TestUtil;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
 
 import com.google.common.collect.Maps;
 
-@RunWith(Parameterized.class)
+
 public class StatsCollectorIT extends StatsCollectorAbstractIT {
     private static final String STATS_TEST_TABLE_NAME = "S";
-    
-    private final String tableDDLOptions;
-    private final String tableName;
-    private final String fullTableName;
         
     @BeforeClass
     public static void doSetup() throws Exception {
@@ -73,31 +62,30 @@ public class StatsCollectorIT extends StatsCollectorAbstractIT {
         props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
         setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
     }
-    
-    public StatsCollectorIT( boolean transactional) {
-        this.tableDDLOptions= transactional ? " TRANSACTIONAL=true" : "";
-        this.tableName = TestUtil.DEFAULT_DATA_TABLE_NAME + ( transactional ?  "_TXN" : "");
-        this.fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+
+    @Test
+    public void testUpdateStatsForNonTxnTable() throws Throwable {
+        helpTestUpdateStats(false);
     }
     
-    @Parameters(name="transactional = {0}")
-    public static Collection<Boolean> data() {
-        return Arrays.asList(false,true);
+    @Test
+    public void testUpdateStatsForTxnTable() throws Throwable {
+        helpTestUpdateStats(true);
     }
 
-    @Test
-    public void testUpdateStats() throws SQLException, IOException,
+	private void helpTestUpdateStats(boolean transactional) throws SQLException, IOException,
 			InterruptedException {
 		Connection conn;
         PreparedStatement stmt;
         ResultSet rs;
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "T" + (transactional ? "_TXN" : "");
         // props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 10));
         conn = DriverManager.getConnection(getUrl(), props);
         conn.createStatement().execute(
-                "CREATE TABLE " + fullTableName +" ( k VARCHAR, a_string_array VARCHAR(100) ARRAY[4], b_string_array VARCHAR(100) ARRAY[4] \n"
-                        + " CONSTRAINT pk PRIMARY KEY (k, b_string_array DESC))"
-                		+ tableDDLOptions );
+                "CREATE TABLE " + tableName +" ( k VARCHAR, a_string_array VARCHAR(100) ARRAY[4], b_string_array VARCHAR(100) ARRAY[4] \n"
+                        + " CONSTRAINT pk PRIMARY KEY (k, b_string_array DESC))" 
+                		+ (transactional ? " TRANSACTIONAL=true" : ""));
         String[] s;
         Array array;
         conn = upsertValues(props, tableName);
@@ -131,14 +119,14 @@ public class StatsCollectorIT extends StatsCollectorAbstractIT {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         conn = DriverManager.getConnection(getUrl(), props);
         conn.createStatement()
-                .execute("CREATE TABLE " + fullTableName +" ( k VARCHAR, c1.a bigint,c2.b bigint CONSTRAINT pk PRIMARY KEY (k))" + tableDDLOptions );
-        conn.createStatement().execute("upsert into " + fullTableName +" values ('abc',1,3)");
-        conn.createStatement().execute("upsert into " + fullTableName +" values ('def',2,4)");
+                .execute("CREATE TABLE x ( k VARCHAR, c1.a bigint,c2.b bigint CONSTRAINT pk PRIMARY KEY (k)) \n");
+        conn.createStatement().execute("upsert into x values ('abc',1,3)");
+        conn.createStatement().execute("upsert into x values ('def',2,4)");
         conn.commit();
         // CAll the update statistics query here
-        stmt = conn.prepareStatement("UPDATE STATISTICS " + fullTableName);
+        stmt = conn.prepareStatement("UPDATE STATISTICS X");
         stmt.execute();
-        rs = conn.createStatement().executeQuery("SELECT k FROM " + fullTableName);
+        rs = conn.createStatement().executeQuery("SELECT k FROM x");
         assertTrue(rs.next());
         assertEquals("abc", rs.getString(1));
         assertTrue(rs.next());
@@ -149,7 +137,6 @@ public class StatsCollectorIT extends StatsCollectorAbstractIT {
 
     @Test
     public void testUpdateStatsWithMultipleTables() throws Throwable {
-        String fullTableName2 = fullTableName+"_2";
         Connection conn;
         PreparedStatement stmt;
         ResultSet rs;
@@ -157,21 +144,21 @@ public class StatsCollectorIT extends StatsCollectorAbstractIT {
         // props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 10));
         conn = DriverManager.getConnection(getUrl(), props);
         conn.createStatement().execute(
-                "CREATE TABLE " + fullTableName +" ( k VARCHAR, a_string_array VARCHAR(100) ARRAY[4], b_string_array VARCHAR(100) ARRAY[4] \n"
-                        + " CONSTRAINT pk PRIMARY KEY (k, b_string_array DESC))" + tableDDLOptions );
+                "CREATE TABLE x ( k VARCHAR, a_string_array VARCHAR(100) ARRAY[4], b_string_array VARCHAR(100) ARRAY[4] \n"
+                        + " CONSTRAINT pk PRIMARY KEY (k, b_string_array DESC)) \n");
         conn.createStatement().execute(
-                "CREATE TABLE " + fullTableName2 +" ( k VARCHAR, a_string_array VARCHAR(100) ARRAY[4], b_string_array VARCHAR(100) ARRAY[4] \n"
-                        + " CONSTRAINT pk PRIMARY KEY (k, b_string_array DESC))" + tableDDLOptions );
+                "CREATE TABLE z ( k VARCHAR, a_string_array VARCHAR(100) ARRAY[4], b_string_array VARCHAR(100) ARRAY[4] \n"
+                        + " CONSTRAINT pk PRIMARY KEY (k, b_string_array DESC)) \n");
         String[] s;
         Array array;
-        conn = upsertValues(props, fullTableName);
-        conn = upsertValues(props, fullTableName2);
+        conn = upsertValues(props, "x");
+        conn = upsertValues(props, "z");
         // CAll the update statistics query here
-        stmt = conn.prepareStatement("UPDATE STATISTICS "+fullTableName);
+        stmt = conn.prepareStatement("UPDATE STATISTICS X");
         stmt.execute();
-        stmt = conn.prepareStatement("UPDATE STATISTICS "+fullTableName2);
+        stmt = conn.prepareStatement("UPDATE STATISTICS Z");
         stmt.execute();
-        stmt = upsertStmt(conn, fullTableName);
+        stmt = upsertStmt(conn, "x");
         stmt.setString(1, "z");
         s = new String[] { "xyz", "def", "ghi", "jkll", null, null, "xxx" };
         array = conn.createArrayOf("VARCHAR", s);
@@ -180,7 +167,7 @@ public class StatsCollectorIT extends StatsCollectorAbstractIT {
         array = conn.createArrayOf("VARCHAR", s);
         stmt.setArray(3, array);
         stmt.execute();
-        stmt = upsertStmt(conn, fullTableName2);
+        stmt = upsertStmt(conn, "z");
         stmt.setString(1, "z");
         s = new String[] { "xyz", "def", "ghi", "jkll", null, null, "xxx" };
         array = conn.createArrayOf("VARCHAR", s);
@@ -192,9 +179,9 @@ public class StatsCollectorIT extends StatsCollectorAbstractIT {
         conn.close();
         conn = DriverManager.getConnection(getUrl(), props);
         // This analyze would not work
-        stmt = conn.prepareStatement("UPDATE STATISTICS "+fullTableName2);
+        stmt = conn.prepareStatement("UPDATE STATISTICS Z");
         stmt.execute();
-        rs = conn.createStatement().executeQuery("SELECT k FROM "+fullTableName2);
+        rs = conn.createStatement().executeQuery("SELECT k FROM Z");
         assertTrue(rs.next());
         conn.close();
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8a799b8c/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
index b65b856..2794c47 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
@@ -20,8 +20,6 @@ package org.apache.phoenix.tx;
 import static org.apache.phoenix.util.TestUtil.INDEX_DATA_SCHEMA;
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
 import static org.apache.phoenix.util.TestUtil.TRANSACTIONAL_DATA_TABLE;
-import static org.apache.phoenix.util.TestUtil.analyzeTable;
-import static org.apache.phoenix.util.TestUtil.getAllSplits;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -38,6 +36,12 @@ import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 
+import co.cask.tephra.TransactionContext;
+import co.cask.tephra.TransactionSystemClient;
+import co.cask.tephra.TxConstants;
+import co.cask.tephra.hbase11.TransactionAwareHTable;
+import co.cask.tephra.hbase11.coprocessor.TransactionProcessor;
+
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
@@ -52,7 +56,6 @@ import org.apache.phoenix.end2end.Shadower;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
-import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
@@ -68,12 +71,6 @@ import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
 
-import co.cask.tephra.TransactionContext;
-import co.cask.tephra.TransactionSystemClient;
-import co.cask.tephra.TxConstants;
-import co.cask.tephra.hbase11.TransactionAwareHTable;
-import co.cask.tephra.hbase11.coprocessor.TransactionProcessor;
-
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
@@ -89,9 +86,8 @@ public class TransactionIT extends BaseHBaseManagedTimeIT {
     @BeforeClass
     @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
     public static void doSetup() throws Exception {
-        Map<String,String> props = Maps.newHashMapWithExpectedSize(2);
+        Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
         props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
-        props.put(QueryServices.STATS_GUIDEPOST_WIDTH_BYTES_ATTRIB, Integer.toString(20));
         setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
     }
         
@@ -585,107 +581,30 @@ public class TransactionIT extends BaseHBaseManagedTimeIT {
     }
     
     @Test
-    public void testReadOnlyViewWithStats() throws Exception {
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
-            String ddl = "CREATE TABLE t (k INTEGER NOT NULL PRIMARY KEY, v1 DATE) TRANSACTIONAL=true";
-            conn.createStatement().execute(ddl);
-            ddl = "CREATE VIEW v (v2 VARCHAR) AS SELECT * FROM t where k>5";
-            conn.createStatement().execute(ddl);
-            for (int i = 0; i < 10; i++) {
-                conn.createStatement().execute("UPSERT INTO t VALUES(" + i + ")");
-            }
-            conn.commit();
-            
-            // verify rows are visible for stats
-            analyzeTable(conn, "v", true);
-            List<KeyRange> splits = getAllSplits(conn, "v");
-            assertEquals(4, splits.size());
-            
-            int count = 0;
-            ResultSet rs = conn.createStatement().executeQuery("SELECT k FROM t");
-            while (rs.next()) {
-                assertEquals(count++, rs.getInt(1));
-            }
-            assertEquals(10, count);
-            
-            count = 0;
-            rs = conn.createStatement().executeQuery("SELECT k FROM v");
-            while (rs.next()) {
-                assertEquals(6+count++, rs.getInt(1));
-            }
-            assertEquals(4, count);
+    public void testReadOnlyView() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        String ddl = "CREATE TABLE t (k INTEGER NOT NULL PRIMARY KEY, v1 DATE) TRANSACTIONAL=true";
+        conn.createStatement().execute(ddl);
+        ddl = "CREATE VIEW v (v2 VARCHAR) AS SELECT * FROM t where k>4";
+        conn.createStatement().execute(ddl);
+        for (int i = 0; i < 10; i++) {
+            conn.createStatement().execute("UPSERT INTO t VALUES(" + i + ")");
         }
-    }
-    
-    @Test
-    public void testReadOwnWritesWithStats() throws Exception {
-        try (Connection conn1 = DriverManager.getConnection(getUrl()); 
-                Connection conn2 = DriverManager.getConnection(getUrl())) {
-            String ddl = "CREATE TABLE t (k INTEGER NOT NULL PRIMARY KEY, v1 DATE) TRANSACTIONAL=true";
-            conn1.createStatement().execute(ddl);
-            ddl = "CREATE VIEW v (v2 VARCHAR) AS SELECT * FROM t where k>5";
-            conn1.createStatement().execute(ddl);
-            for (int i = 0; i < 10; i++) {
-                conn1.createStatement().execute("UPSERT INTO t VALUES(" + i + ")");
-            }
-    
-            // verify you can read your own writes
-            int count = 0;
-            ResultSet rs = conn1.createStatement().executeQuery("SELECT k FROM t");
-            while (rs.next()) {
-                assertEquals(count++, rs.getInt(1));
-            }
-            assertEquals(10, count);
-            
-            count = 0;
-            rs = conn1.createStatement().executeQuery("SELECT k FROM v");
-            while (rs.next()) {
-                assertEquals(6+count++, rs.getInt(1));
-            }
-            assertEquals(4, count);
-            
-            // verify stats can see the read own writes rows
-            analyzeTable(conn2, "v", true);
-            List<KeyRange> splits = getAllSplits(conn2, "v");
-            assertEquals(4, splits.size());
+        conn.commit();
+        
+        int count = 0;
+        ResultSet rs = conn.createStatement().executeQuery("SELECT k FROM t");
+        while (rs.next()) {
+            assertEquals(count++, rs.getInt(1));
         }
-    }
-    
-    @Test
-    public void testInvalidRowsWithStats() throws Exception {
-        try (Connection conn1 = DriverManager.getConnection(getUrl()); 
-                Connection conn2 = DriverManager.getConnection(getUrl())) {
-            String ddl = "CREATE TABLE t (k INTEGER NOT NULL PRIMARY KEY, v1 DATE) TRANSACTIONAL=true";
-            conn1.createStatement().execute(ddl);
-            ddl = "CREATE VIEW v (v2 VARCHAR) AS SELECT * FROM t where k>5";
-            conn1.createStatement().execute(ddl);
-            for (int i = 0; i < 10; i++) {
-                conn1.createStatement().execute("UPSERT INTO t VALUES(" + i + ")");
-            }
-    
-            // verify you can read your own writes
-            int count = 0;
-            ResultSet rs = conn1.createStatement().executeQuery("SELECT k FROM t");
-            while (rs.next()) {
-                assertEquals(count++, rs.getInt(1));
-            }
-            assertEquals(10, count);
-            
-            count = 0;
-            rs = conn1.createStatement().executeQuery("SELECT k FROM v");
-            while (rs.next()) {
-                assertEquals(6+count++, rs.getInt(1));
-            }
-            assertEquals(4, count);
-            
-            Thread.sleep(DEFAULT_TXN_TIMEOUT_SECONDS*1000+20000);
-            assertEquals("There should be one invalid transaction", 1, txManager.getInvalidSize());
-            
-            // verify stats can see the rows from the invalid transaction
-            analyzeTable(conn2, "v", true);
-            List<KeyRange> splits = getAllSplits(conn2, "v");
-            assertEquals(4, splits.size());
+        assertEquals(10, count);
+        
+        count = 0;
+        rs = conn.createStatement().executeQuery("SELECT k FROM v");
+        while (rs.next()) {
+            assertEquals(5+count++, rs.getInt(1));
         }
+        assertEquals(5, count);
     }
     
     @Test


[20/50] [abbrv] phoenix git commit: PHOENIX-2616 Indexes over immutable tables not marked as immutable

Posted by ma...@apache.org.
PHOENIX-2616 Indexes over immutable tables not marked as immutable


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

Branch: refs/heads/calcite
Commit: d345b8802590612e69181c5fcb093dce69143b67
Parents: 7795f80
Author: James Taylor <jt...@salesforce.com>
Authored: Wed Jan 20 21:04:03 2016 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Wed Jan 20 21:04:03 2016 -0800

----------------------------------------------------------------------
 .../src/it/java/org/apache/phoenix/end2end/index/IndexIT.java    | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/d345b880/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
index e369dae..6d54076 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
@@ -70,6 +70,7 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
 	
 	private final boolean localIndex;
     private final boolean transactional;
+    private final boolean mutable;
 	private final String tableDDLOptions;
 	private final String tableName;
     private final String indexName;
@@ -79,6 +80,7 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
 	public IndexIT(boolean localIndex, boolean mutable, boolean transactional) {
 		this.localIndex = localIndex;
 		this.transactional = transactional;
+		this.mutable = mutable;
 		StringBuilder optionBuilder = new StringBuilder();
 		if (!mutable) 
 			optionBuilder.append(" IMMUTABLE_ROWS=true ");
@@ -461,7 +463,7 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
 	        // Can't set IMMUTABLE_ROWS, MULTI_TENANT or DEFAULT_COLUMN_FAMILY_NAME on an index
 	        assertNull(indexTable.getDefaultFamilyName());
 	        assertFalse(indexTable.isMultiTenant());
-	        assertFalse(indexTable.isImmutableRows());
+	        assertEquals(mutable, !indexTable.isImmutableRows()); // Should match table
 	        if(localIndex) {
 	            assertEquals(10, indexTable.getBucketNum().intValue());
 	            assertTrue(indexTable.isWALDisabled());


[42/50] [abbrv] phoenix git commit: PHOENIX-1849 - MemoryLeak in PhoenixFlumePlugin PhoenixConnection

Posted by ma...@apache.org.
PHOENIX-1849 - MemoryLeak in PhoenixFlumePlugin PhoenixConnection


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

Branch: refs/heads/calcite
Commit: ee6370200b491bd210dd4f89d6be59162c994bd4
Parents: c9594e8
Author: ravimagham <ra...@bazaarvoice.com>
Authored: Thu Jan 28 13:43:33 2016 -0800
Committer: ravimagham <ra...@bazaarvoice.com>
Committed: Thu Jan 28 13:43:33 2016 -0800

----------------------------------------------------------------------
 .../phoenix/flume/serializer/RegexEventSerializer.java  | 12 +++++-------
 1 file changed, 5 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/ee637020/phoenix-flume/src/main/java/org/apache/phoenix/flume/serializer/RegexEventSerializer.java
----------------------------------------------------------------------
diff --git a/phoenix-flume/src/main/java/org/apache/phoenix/flume/serializer/RegexEventSerializer.java b/phoenix-flume/src/main/java/org/apache/phoenix/flume/serializer/RegexEventSerializer.java
index 6562eb7..b636481 100644
--- a/phoenix-flume/src/main/java/org/apache/phoenix/flume/serializer/RegexEventSerializer.java
+++ b/phoenix-flume/src/main/java/org/apache/phoenix/flume/serializer/RegexEventSerializer.java
@@ -31,11 +31,11 @@ import java.util.regex.Pattern;
 
 import org.apache.flume.Context;
 import org.apache.flume.Event;
+import org.apache.phoenix.schema.types.PDataType;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Preconditions;
-import org.apache.phoenix.schema.types.PDataType;
 
 public class RegexEventSerializer extends BaseEventSerializer {
 
@@ -69,13 +69,11 @@ public class RegexEventSerializer extends BaseEventSerializer {
        Preconditions.checkNotNull(connection);
        Preconditions.checkNotNull(this.upsertStatement);
        
-       PreparedStatement colUpsert = connection.prepareStatement(upsertStatement);
        boolean wasAutoCommit = connection.getAutoCommit();
        connection.setAutoCommit(false);
-       
-       String value = null;
-       Integer sqlType = null;
-       try {
+       try (PreparedStatement colUpsert = connection.prepareStatement(upsertStatement)) {
+           String value = null;
+           Integer sqlType = null;
            for(Event event : events) {
                byte [] payloadBytes = event.getBody();
                if(payloadBytes == null || payloadBytes.length == 0) {
@@ -136,7 +134,7 @@ public class RegexEventSerializer extends BaseEventSerializer {
        } catch(Exception ex){
            logger.error("An error {} occurred during persisting the event ",ex.getMessage());
            throw new SQLException(ex.getMessage());
-       }finally {
+       } finally {
            if(wasAutoCommit) {
                connection.setAutoCommit(true);
            }


[35/50] [abbrv] phoenix git commit: PHOENIX-2348 The value of DEFAULT_SPOOL_DIRECTORY is invalid on Windows (ChiaPing Tsai)

Posted by ma...@apache.org.
PHOENIX-2348 The value of DEFAULT_SPOOL_DIRECTORY is invalid on Windows (ChiaPing Tsai)


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

Branch: refs/heads/calcite
Commit: 5176ed6371778daa58300592620483b619b668fb
Parents: e2b6009
Author: Dumindu Buddhika <du...@apache.org>
Authored: Tue Jan 26 08:29:15 2016 +0530
Committer: Dumindu Buddhika <du...@apache.org>
Committed: Tue Jan 26 08:29:15 2016 +0530

----------------------------------------------------------------------
 .../main/java/org/apache/phoenix/query/QueryServicesOptions.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/5176ed63/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 c9bc19b..1838b51 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
@@ -103,7 +103,7 @@ public class QueryServicesOptions {
 	public static final int DEFAULT_QUEUE_SIZE = 5000;
 	public static final int DEFAULT_THREAD_TIMEOUT_MS = 600000; // 10min
 	public static final int DEFAULT_SPOOL_THRESHOLD_BYTES = 1024 * 1024 * 20; // 20m
-    public static final String DEFAULT_SPOOL_DIRECTORY = "/tmp";
+    public static final String DEFAULT_SPOOL_DIRECTORY = System.getProperty("java.io.tmpdir");
 	public static final int DEFAULT_MAX_MEMORY_PERC = 15; // 15% of heap
 	public static final int DEFAULT_MAX_MEMORY_WAIT_MS = 10000;
 	public static final int DEFAULT_MAX_TENANT_MEMORY_PERC = 100;


[44/50] [abbrv] phoenix git commit: PHOENIX-2629 NoClassDef error for BaseDecoder on log replay

Posted by ma...@apache.org.
PHOENIX-2629 NoClassDef error for BaseDecoder on log replay


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

Branch: refs/heads/calcite
Commit: c89903ec5c176eb93abe32437b2ac171b6f6c552
Parents: 791a27c
Author: Samarth <sa...@salesforce.com>
Authored: Thu Jan 28 13:30:54 2016 -0800
Committer: Samarth <sa...@salesforce.com>
Committed: Thu Jan 28 15:01:01 2016 -0800

----------------------------------------------------------------------
 .../wal/BinaryCompatibleBaseDecoder.java        | 110 +++++++++++++++++++
 .../regionserver/wal/IndexedWALEditCodec.java   |  83 +++++++++++++-
 2 files changed, 191 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/c89903ec/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/wal/BinaryCompatibleBaseDecoder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/wal/BinaryCompatibleBaseDecoder.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/wal/BinaryCompatibleBaseDecoder.java
new file mode 100644
index 0000000..80f2dd2
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/wal/BinaryCompatibleBaseDecoder.java
@@ -0,0 +1,110 @@
+/**
+ * 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.hadoop.hbase.regionserver.wal;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.PushbackInputStream;
+
+import javax.annotation.Nonnull;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.codec.Codec;
+
+/**
+ * This class is a copy paste version of org.apache.hadoop.hbase.codec.BaseDecoder class. 
+ * This class is meant to be used in {@link IndexedWALEditCodec} when runtime version of
+ * HBase is older than 1.1.3. This is needed to handle binary incompatibility introduced by
+ * HBASE-14501. See PHOENIX-2629 and PHOENIX-2636 for details.
+ */
+public abstract class BinaryCompatibleBaseDecoder implements Codec.Decoder {
+  protected static final Log LOG = LogFactory.getLog(BinaryCompatibleBaseDecoder.class);
+
+  protected final InputStream in;
+  private Cell current = null;
+
+  protected static class PBIS extends PushbackInputStream {
+    public PBIS(InputStream in, int size) {
+      super(in, size);
+    }
+
+    public void resetBuf(int size) {
+      this.buf = new byte[size];
+      this.pos = size;
+    }
+  }
+
+  public BinaryCompatibleBaseDecoder(final InputStream in) {
+    this.in = new PBIS(in, 1);
+  }
+
+  @Override
+  public boolean advance() throws IOException {
+    int firstByte = in.read();
+    if (firstByte == -1) {
+      return false;
+    } else {
+      ((PBIS)in).unread(firstByte);
+    }
+
+    try {
+      this.current = parseCell();
+    } catch (IOException ioEx) {
+      ((PBIS)in).resetBuf(1); // reset the buffer in case the underlying stream is read from upper layers
+      rethrowEofException(ioEx);
+    }
+    return true;
+  }
+
+  private void rethrowEofException(IOException ioEx) throws IOException {
+    boolean isEof = false;
+    try {
+      isEof = this.in.available() == 0;
+    } catch (Throwable t) {
+      LOG.trace("Error getting available for error message - ignoring", t);
+    }
+    if (!isEof) throw ioEx;
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Partial cell read caused by EOF", ioEx);
+    }
+    EOFException eofEx = new EOFException("Partial cell read");
+    eofEx.initCause(ioEx);
+    throw eofEx;
+  }
+
+  protected InputStream getInputStream() {
+    return in;
+  }
+
+  /**
+   * Extract a Cell.
+   * @return a parsed Cell or throws an Exception. EOFException or a generic IOException maybe
+   * thrown if EOF is reached prematurely. Does not return null.
+   * @throws IOException
+   */
+  @Nonnull
+  protected abstract Cell parseCell() throws IOException;
+
+  @Override
+  public Cell current() {
+    return this.current;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c89903ec/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/wal/IndexedWALEditCodec.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/wal/IndexedWALEditCodec.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/wal/IndexedWALEditCodec.java
index 2534b34..1a70e12 100644
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/wal/IndexedWALEditCodec.java
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/wal/IndexedWALEditCodec.java
@@ -33,6 +33,8 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.codec.BaseDecoder;
 import org.apache.hadoop.hbase.codec.BaseEncoder;
+import org.apache.hadoop.hbase.util.VersionInfo;
+import org.apache.phoenix.hbase.index.util.VersionUtil;
 import org.apache.phoenix.hbase.index.wal.IndexedKeyValue;
 import org.apache.phoenix.hbase.index.wal.KeyValueCodec;
 
@@ -50,23 +52,27 @@ public class IndexedWALEditCodec extends WALCellCodec {
   // the stream
   private static final int REGULAR_KEY_VALUE_MARKER = 0;
   private CompressionContext compression;
+  private static final int MIN_BINARY_COMPATIBLE_INDEX_CODEC_VERSION = VersionUtil.encodeVersion("1", "1", "3");
+  private final boolean useDefaultDecoder;
 
   public IndexedWALEditCodec(Configuration conf, CompressionContext compression) {
       super(conf, compression);
       this.compression = compression;
+      String hbaseVersion = VersionInfo.getVersion();
+      this.useDefaultDecoder = VersionUtil.encodeVersion(hbaseVersion) >= MIN_BINARY_COMPATIBLE_INDEX_CODEC_VERSION;
   }
 
   @Override
   public Decoder getDecoder(InputStream is) {
     // compression isn't enabled
     if (this.compression == null) {
-      return new IndexKeyValueDecoder(is);
+      return useDefaultDecoder ? new IndexKeyValueDecoder(is) : new BinaryCompatibleIndexKeyValueDecoder(is);
     }
 
     // there is compression, so we get the standard decoder to handle reading those kvs
     Decoder decoder = super.getDecoder(is);
     // compression is on, reqturn our custom decoder
-    return new CompressedIndexKeyValueDecoder(is, decoder);
+    return useDefaultDecoder ? new CompressedIndexKeyValueDecoder(is, decoder) : new BinaryCompatibleCompressedIndexKeyValueDecoder(is, decoder);
   }
 
   @Override
@@ -237,4 +243,77 @@ public class IndexedWALEditCodec extends WALCellCodec {
       }
     }
   }
+  
+  private static abstract class BinaryCompatiblePhoenixBaseDecoder extends BinaryCompatibleBaseDecoder {
+      protected DataInput dataInput;
+      public BinaryCompatiblePhoenixBaseDecoder(InputStream in) {
+        super(in);
+        dataInput = getDataInput(this.in);
+      } 
+  }
+  
+  /**
+   * This class is meant to be used when runtime version of HBase
+   * HBase is older than 1.1.3. This is needed to handle binary incompatibility introduced by
+   * HBASE-14501. See PHOENIX-2629 and PHOENIX-2636 for details.
+   */
+  private static class BinaryCompatibleIndexKeyValueDecoder extends BinaryCompatiblePhoenixBaseDecoder {
+      /**
+       * Create a Decoder on the given input stream with the given Decoder to parse
+       * generic {@link KeyValue}s.
+       * @param is stream to read from
+       */
+      public BinaryCompatibleIndexKeyValueDecoder(InputStream is){
+        super(is);
+      }
+
+      @Override
+      protected KeyValue parseCell() throws IOException{
+        return KeyValueCodec.readKeyValue(this.dataInput);
+      }
+  }
+  
+  /**
+   * This class is meant to be used when runtime version of HBase
+   * HBase is older than 1.1.3. This is needed to handle binary incompatibility introduced by
+   * HBASE-14501. See PHOENIX-2629 and PHOENIX-2636 for details.
+   */
+  private static class BinaryCompatibleCompressedIndexKeyValueDecoder extends BinaryCompatiblePhoenixBaseDecoder {
+
+      private Decoder decoder;
+
+      /**
+       * Create a Decoder on the given input stream with the given Decoder to parse
+       * generic {@link KeyValue}s.
+       * @param is stream to read from
+       * @param compressedDecoder decoder for generic {@link KeyValue}s. Should support the expected
+       *          compression.
+       */
+      public BinaryCompatibleCompressedIndexKeyValueDecoder(InputStream is, Decoder compressedDecoder) {
+        super(is);
+        this.decoder = compressedDecoder;
+      }
+
+      @Override
+      protected Cell parseCell() throws IOException {
+        // reader the marker
+        int marker = this.in.read();
+        if (marker < 0) {
+          throw new EOFException(
+              "Unexepcted end of stream found while reading next (Indexed) KeyValue");
+        }
+
+        // do the normal thing, if its a regular kv
+        if (marker == REGULAR_KEY_VALUE_MARKER) {
+          if (!this.decoder.advance()) {
+            throw new IOException("Could not read next key-value from generic KeyValue Decoder!");
+          }
+          return this.decoder.current();
+        }
+
+        // its an indexedKeyValue, so parse it out specially
+        return KeyValueCodec.readKeyValue(this.dataInput);
+      }
+  }
+  
 }
\ No newline at end of file


[11/50] [abbrv] phoenix git commit: PHOENIX-2610 Allow passing PHOENIX_OPTS to command line utils

Posted by ma...@apache.org.
PHOENIX-2610 Allow passing PHOENIX_OPTS to command line utils


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

Branch: refs/heads/calcite
Commit: 588f449b0bfc5ba6398c30ee76745a78c4996976
Parents: 0b7a3c4
Author: Nick Dimiduk <nd...@apache.org>
Authored: Tue Jan 19 20:17:55 2016 -0800
Committer: Nick Dimiduk <nd...@apache.org>
Committed: Tue Jan 19 20:50:37 2016 -0800

----------------------------------------------------------------------
 bin/performance.py     | 2 +-
 bin/phoenix_sandbox.py | 2 +-
 bin/psql.py            | 3 ++-
 bin/queryserver.py     | 2 +-
 bin/sqlline-thin.py    | 3 ++-
 bin/sqlline.py         | 3 ++-
 bin/traceserver.py     | 3 ++-
 7 files changed, 11 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/588f449b/bin/performance.py
----------------------------------------------------------------------
diff --git a/bin/performance.py b/bin/performance.py
index fe39ddf..c16dd5a 100755
--- a/bin/performance.py
+++ b/bin/performance.py
@@ -96,7 +96,7 @@ if java_home:
 else:
     java_cmd = 'java'
 
-execute = ('%s -cp "%s%s%s" -Dlog4j.configuration=file:' +
+execute = ('%s $PHOENIX_OPTS -cp "%s%s%s" -Dlog4j.configuration=file:' +
            os.path.join(phoenix_utils.current_dir, "log4j.properties") +
            ' org.apache.phoenix.util.PhoenixRuntime -t %s %s ') % \
     (java_cmd, hbase_config_path, os.pathsep, phoenix_utils.phoenix_client_jar, table, zookeeper)

http://git-wip-us.apache.org/repos/asf/phoenix/blob/588f449b/bin/phoenix_sandbox.py
----------------------------------------------------------------------
diff --git a/bin/phoenix_sandbox.py b/bin/phoenix_sandbox.py
index 433bc98..4279dd6 100755
--- a/bin/phoenix_sandbox.py
+++ b/bin/phoenix_sandbox.py
@@ -43,7 +43,7 @@ cp_components = [phoenix_target_dir + "/*"]
 with open(cp_file_path, 'rb') as cp_file:
     cp_components.append(cp_file.read())
 
-java_cmd = ("java -Dlog4j.configuration=file:%s " +
+java_cmd = ("java $PHOENIX_OPTS -Dlog4j.configuration=file:%s " +
                 "-cp %s org.apache.phoenix.Sandbox") % (
                             logging_config, ":".join(cp_components))
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/588f449b/bin/psql.py
----------------------------------------------------------------------
diff --git a/bin/psql.py b/bin/psql.py
index d4269d1..973d3de 100755
--- a/bin/psql.py
+++ b/bin/psql.py
@@ -62,7 +62,8 @@ if java_home:
 else:
     java = 'java'
 
-java_cmd = java +' -cp "' + phoenix_utils.hbase_conf_dir + os.pathsep + phoenix_utils.phoenix_client_jar + \
+java_cmd = java + ' $PHOENIX_OPTS ' + \
+    ' -cp "' + phoenix_utils.hbase_conf_dir + os.pathsep + phoenix_utils.phoenix_client_jar + \
     os.pathsep + phoenix_utils.hadoop_conf + os.pathsep + phoenix_utils.hadoop_classpath + '" -Dlog4j.configuration=file:' + \
     os.path.join(phoenix_utils.current_dir, "log4j.properties") + \
     " org.apache.phoenix.util.PhoenixRuntime " + args 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/588f449b/bin/queryserver.py
----------------------------------------------------------------------
diff --git a/bin/queryserver.py b/bin/queryserver.py
index 78d340c..d4228b3 100755
--- a/bin/queryserver.py
+++ b/bin/queryserver.py
@@ -119,7 +119,7 @@ else:
 
 #    " -Xdebug -Xrunjdwp:transport=dt_socket,address=5005,server=y,suspend=n " + \
 #    " -XX:+UnlockCommercialFeatures -XX:+FlightRecorder -XX:FlightRecorderOptions=defaultrecording=true,dumponexit=true" + \
-java_cmd = '%(java)s -cp ' + hbase_config_path + os.pathsep + hadoop_config_path + os.pathsep + \
+java_cmd = '%(java)s $PHOENIX_OPTS -cp ' + hbase_config_path + os.pathsep + hadoop_config_path + os.pathsep + \
     phoenix_utils.phoenix_queryserver_jar + os.pathsep + phoenix_utils.phoenix_client_jar + \
     " -Dproc_phoenixserver" + \
     " -Dlog4j.configuration=file:" + os.path.join(phoenix_utils.current_dir, "log4j.properties") + \

http://git-wip-us.apache.org/repos/asf/phoenix/blob/588f449b/bin/sqlline-thin.py
----------------------------------------------------------------------
diff --git a/bin/sqlline-thin.py b/bin/sqlline-thin.py
index d17d756..b37cd90 100755
--- a/bin/sqlline-thin.py
+++ b/bin/sqlline-thin.py
@@ -145,7 +145,8 @@ if java_home:
 else:
     java = 'java'
 
-java_cmd = java + ' -cp "' + phoenix_utils.hbase_conf_dir + os.pathsep + phoenix_utils.phoenix_thin_client_jar + \
+java_cmd = java + ' $PHOENIX_OPTS ' + \
+    ' -cp "' + phoenix_utils.hbase_conf_dir + os.pathsep + phoenix_utils.phoenix_thin_client_jar + \
     os.pathsep + phoenix_utils.hadoop_conf + os.pathsep + phoenix_utils.hadoop_classpath + '" -Dlog4j.configuration=file:' + \
     os.path.join(phoenix_utils.current_dir, "log4j.properties") + \
     " sqlline.SqlLine -d org.apache.phoenix.queryserver.client.Driver " + \

http://git-wip-us.apache.org/repos/asf/phoenix/blob/588f449b/bin/sqlline.py
----------------------------------------------------------------------
diff --git a/bin/sqlline.py b/bin/sqlline.py
index d1fdf74..f14cfc7 100755
--- a/bin/sqlline.py
+++ b/bin/sqlline.py
@@ -87,7 +87,8 @@ colorSetting = "true"
 if os.name == 'nt':
     colorSetting = "false"
 
-java_cmd = java + ' -cp "' + phoenix_utils.hbase_conf_dir + os.pathsep + phoenix_utils.phoenix_client_jar + os.pathsep + phoenix_utils.hadoop_common_jar + os.pathsep + phoenix_utils.hadoop_hdfs_jar + \
+java_cmd = java + ' $PHOENIX_OPTS ' + \
+    ' -cp "' + phoenix_utils.hbase_conf_dir + os.pathsep + phoenix_utils.phoenix_client_jar + os.pathsep + phoenix_utils.hadoop_common_jar + os.pathsep + phoenix_utils.hadoop_hdfs_jar + \
     os.pathsep + phoenix_utils.hadoop_conf + os.pathsep + phoenix_utils.hadoop_classpath + '" -Dlog4j.configuration=file:' + \
     os.path.join(phoenix_utils.current_dir, "log4j.properties") + \
     " sqlline.SqlLine -d org.apache.phoenix.jdbc.PhoenixDriver \

http://git-wip-us.apache.org/repos/asf/phoenix/blob/588f449b/bin/traceserver.py
----------------------------------------------------------------------
diff --git a/bin/traceserver.py b/bin/traceserver.py
index 082b417..665099e 100755
--- a/bin/traceserver.py
+++ b/bin/traceserver.py
@@ -116,7 +116,8 @@ else:
 
 #    " -Xdebug -Xrunjdwp:transport=dt_socket,address=5005,server=y,suspend=n " + \
 #    " -XX:+UnlockCommercialFeatures -XX:+FlightRecorder -XX:FlightRecorderOptions=defaultrecording=true,dumponexit=true" + \
-java_cmd = '%(java)s -cp ' + hbase_config_path + os.pathsep + phoenix_utils.phoenix_traceserver_jar + os.pathsep + phoenix_utils.phoenix_client_jar + \
+java_cmd = '%(java)s $PHOENIX_OPTS ' + \
+    '-cp ' + hbase_config_path + os.pathsep + phoenix_utils.phoenix_traceserver_jar + os.pathsep + phoenix_utils.phoenix_client_jar + \
     " -Dproc_phoenixtraceserver" + \
     " -Dlog4j.configuration=file:" + os.path.join(phoenix_utils.current_dir, "log4j.properties") + \
     " -Dpsql.root.logger=%(root_logger)s" + \


[33/50] [abbrv] phoenix git commit: Set version to 4.7.0-HBase-1.1 for release

Posted by ma...@apache.org.
Set version to 4.7.0-HBase-1.1 for release


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

Branch: refs/heads/calcite
Commit: 551cc7db93a8a2c3cc9ff15e7cf9425e311ab125
Parents: 424083d
Author: Mujtaba <mu...@apache.org>
Authored: Fri Jan 22 22:29:53 2016 -0800
Committer: Mujtaba <mu...@apache.org>
Committed: Fri Jan 22 22:29:53 2016 -0800

----------------------------------------------------------------------
 phoenix-assembly/pom.xml       | 2 +-
 phoenix-core/pom.xml           | 2 +-
 phoenix-flume/pom.xml          | 2 +-
 phoenix-pherf/pom.xml          | 2 +-
 phoenix-pig/pom.xml            | 2 +-
 phoenix-server-client/pom.xml  | 2 +-
 phoenix-server/pom.xml         | 2 +-
 phoenix-spark/pom.xml          | 2 +-
 phoenix-tracing-webapp/pom.xml | 2 +-
 pom.xml                        | 2 +-
 10 files changed, 10 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/551cc7db/phoenix-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-assembly/pom.xml b/phoenix-assembly/pom.xml
index ab8f42b..a598e61 100644
--- a/phoenix-assembly/pom.xml
+++ b/phoenix-assembly/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.7.0-HBase-1.1-SNAPSHOT</version>
+    <version>4.7.0-HBase-1.1</version>
   </parent>
   <artifactId>phoenix-assembly</artifactId>
   <name>Phoenix Assembly</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/551cc7db/phoenix-core/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-core/pom.xml b/phoenix-core/pom.xml
index c57f5d5..2b6d287 100644
--- a/phoenix-core/pom.xml
+++ b/phoenix-core/pom.xml
@@ -4,7 +4,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.7.0-HBase-1.1-SNAPSHOT</version>
+    <version>4.7.0-HBase-1.1</version>
   </parent>
   <artifactId>phoenix-core</artifactId>
   <name>Phoenix Core</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/551cc7db/phoenix-flume/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-flume/pom.xml b/phoenix-flume/pom.xml
index 24290ac..15a921b 100644
--- a/phoenix-flume/pom.xml
+++ b/phoenix-flume/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.7.0-HBase-1.1-SNAPSHOT</version>
+    <version>4.7.0-HBase-1.1</version>
   </parent>
   <artifactId>phoenix-flume</artifactId>
   <name>Phoenix - Flume</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/551cc7db/phoenix-pherf/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-pherf/pom.xml b/phoenix-pherf/pom.xml
index 568a47f..b94828a 100644
--- a/phoenix-pherf/pom.xml
+++ b/phoenix-pherf/pom.xml
@@ -15,7 +15,7 @@
 	<parent>
 		<groupId>org.apache.phoenix</groupId>
 		<artifactId>phoenix</artifactId>
-		<version>4.7.0-HBase-1.1-SNAPSHOT</version>
+		<version>4.7.0-HBase-1.1</version>
 	</parent>
 
 	<artifactId>phoenix-pherf</artifactId>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/551cc7db/phoenix-pig/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-pig/pom.xml b/phoenix-pig/pom.xml
index c579436..fcaa489 100644
--- a/phoenix-pig/pom.xml
+++ b/phoenix-pig/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.7.0-HBase-1.1-SNAPSHOT</version>
+    <version>4.7.0-HBase-1.1</version>
   </parent>
   <artifactId>phoenix-pig</artifactId>
   <name>Phoenix - Pig</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/551cc7db/phoenix-server-client/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-server-client/pom.xml b/phoenix-server-client/pom.xml
index dae204e..8bff95c 100644
--- a/phoenix-server-client/pom.xml
+++ b/phoenix-server-client/pom.xml
@@ -4,7 +4,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.7.0-HBase-1.1-SNAPSHOT</version>
+    <version>4.7.0-HBase-1.1</version>
   </parent>
   <artifactId>phoenix-server-client</artifactId>
   <name>Phoenix Query Server Client</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/551cc7db/phoenix-server/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-server/pom.xml b/phoenix-server/pom.xml
index 98a771b..36bf726 100644
--- a/phoenix-server/pom.xml
+++ b/phoenix-server/pom.xml
@@ -4,7 +4,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.7.0-HBase-1.1-SNAPSHOT</version>
+    <version>4.7.0-HBase-1.1</version>
   </parent>
   <artifactId>phoenix-server</artifactId>
   <name>Phoenix Query Server</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/551cc7db/phoenix-spark/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-spark/pom.xml b/phoenix-spark/pom.xml
index f82e9a2..4091afa 100644
--- a/phoenix-spark/pom.xml
+++ b/phoenix-spark/pom.xml
@@ -28,7 +28,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.7.0-HBase-1.1-SNAPSHOT</version>
+    <version>4.7.0-HBase-1.1</version>
   </parent>
   <artifactId>phoenix-spark</artifactId>
   <name>Phoenix - Spark</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/551cc7db/phoenix-tracing-webapp/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-tracing-webapp/pom.xml b/phoenix-tracing-webapp/pom.xml
index 08b785f..63edaf4 100755
--- a/phoenix-tracing-webapp/pom.xml
+++ b/phoenix-tracing-webapp/pom.xml
@@ -27,7 +27,7 @@
     <parent>
       <groupId>org.apache.phoenix</groupId>
       <artifactId>phoenix</artifactId>
-      <version>4.7.0-HBase-1.1-SNAPSHOT</version>
+      <version>4.7.0-HBase-1.1</version>
     </parent>
 
     <artifactId>phoenix-tracing-webapp</artifactId>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/551cc7db/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 02639b5..388df2e 100644
--- a/pom.xml
+++ b/pom.xml
@@ -3,7 +3,7 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>org.apache.phoenix</groupId>
   <artifactId>phoenix</artifactId>
-  <version>4.7.0-HBase-1.1-SNAPSHOT</version>
+  <version>4.7.0-HBase-1.1</version>
   <packaging>pom</packaging>
   <name>Apache Phoenix</name>
   <description>A SQL layer over HBase</description>


[28/50] [abbrv] phoenix git commit: PHOENIX-2624 Update pom to Calcite 1.6

Posted by ma...@apache.org.
PHOENIX-2624 Update pom to Calcite 1.6


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

Branch: refs/heads/calcite
Commit: 9e90fabaee2378196c83c9d9ae9aaff332833b16
Parents: 79d109e
Author: James Taylor <jt...@salesforce.com>
Authored: Fri Jan 22 12:03:55 2016 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Fri Jan 22 12:03:55 2016 -0800

----------------------------------------------------------------------
 pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/9e90faba/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index e635ae7..02639b5 100644
--- a/pom.xml
+++ b/pom.xml
@@ -109,7 +109,7 @@
     <collections.version>3.2.1</collections.version>
     <jodatime.version>2.7</jodatime.version>
     <joni.version>2.1.2</joni.version>
-    <calcite.version>1.5.0</calcite.version>
+    <calcite.version>1.6.0</calcite.version>
     <jettyVersion>8.1.7.v20120910</jettyVersion>
     <tephra.version>0.6.4</tephra.version>
 


[47/50] [abbrv] phoenix git commit: Fix merge errors

Posted by ma...@apache.org.
Fix merge errors


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

Branch: refs/heads/calcite
Commit: beb8b81341b4d28585c3cca60b9af40f24a00919
Parents: 51431a2 4625096
Author: maryannxue <ma...@gmail.com>
Authored: Fri Jan 29 22:16:50 2016 -0500
Committer: maryannxue <ma...@gmail.com>
Committed: Fri Jan 29 22:16:50 2016 -0500

----------------------------------------------------------------------
 bin/performance.py                              |   36 +-
 bin/phoenix_sandbox.py                          |    2 +-
 bin/psql.py                                     |    3 +-
 bin/queryserver.py                              |    2 +-
 bin/sqlline-thin.py                             |    3 +-
 bin/sqlline.py                                  |    3 +-
 bin/traceserver.py                              |    3 +-
 .../org/apache/phoenix/end2end/BaseViewIT.java  |   60 +-
 .../org/apache/phoenix/end2end/DateTimeIT.java  |   10 +
 .../apache/phoenix/end2end/SkipScanQueryIT.java |   22 +
 .../apache/phoenix/end2end/UpsertSelectIT.java  |   31 +
 .../java/org/apache/phoenix/end2end/ViewIT.java |   36 +-
 .../index/GlobalIndexOptimizationIT.java        |   23 +
 .../phoenix/end2end/index/ImmutableIndexIT.java |  239 +++-
 .../end2end/index/IndexExpressionIT.java        |   16 +-
 .../apache/phoenix/end2end/index/IndexIT.java   |    4 +-
 .../org/apache/phoenix/rpc/UpdateCacheIT.java   |   92 +-
 .../phoenix/rpc/UpdateCacheWithScnIT.java       |   17 +-
 .../org/apache/phoenix/tx/TransactionIT.java    |  442 +++---
 phoenix-core/src/main/antlr3/PhoenixSQL.g       |    3 +-
 .../apache/phoenix/compile/DeleteCompiler.java  |    1 -
 .../apache/phoenix/compile/FromCompiler.java    |    2 +-
 .../apache/phoenix/compile/JoinCompiler.java    |    2 +-
 .../apache/phoenix/compile/PostDDLCompiler.java |    2 +-
 .../phoenix/compile/PostIndexDDLCompiler.java   |    2 +-
 .../compile/TupleProjectionCompiler.java        |    5 +-
 .../apache/phoenix/compile/UnionCompiler.java   |    2 +-
 .../coprocessor/BaseScannerRegionObserver.java  |   12 +-
 .../coprocessor/MetaDataEndpointImpl.java       |   13 +-
 .../phoenix/coprocessor/MetaDataProtocol.java   |    2 +-
 .../coprocessor/MetaDataRegionObserver.java     |   71 +-
 .../UngroupedAggregateRegionObserver.java       |    2 +-
 .../generated/PGuidePostsProtos.java            |  287 +++-
 .../coprocessor/generated/PTableProtos.java     |  104 +-
 .../generated/StatCollectorProtos.java          | 1269 ------------------
 .../phoenix/exception/SQLExceptionCode.java     |  108 +-
 .../apache/phoenix/execute/MutationState.java   |  636 +++++----
 .../org/apache/phoenix/execute/ScanPlan.java    |    2 +-
 .../expression/function/CeilFunction.java       |    2 +
 .../expression/function/FloorFunction.java      |    4 +-
 .../expression/function/NowFunction.java        |    2 +
 .../expression/function/RoundFunction.java      |    4 +-
 .../expression/function/ToCharFunction.java     |    4 +
 .../expression/function/ToDateFunction.java     |    4 +
 .../expression/function/ToNumberFunction.java   |    4 +
 .../expression/function/ToTimeFunction.java     |    4 +
 .../function/ToTimestampFunction.java           |    4 +
 .../expression/function/TruncFunction.java      |    2 +
 .../apache/phoenix/filter/SkipScanFilter.java   |   24 +-
 .../phoenix/iterate/BaseResultIterators.java    |  138 +-
 .../apache/phoenix/jdbc/PhoenixConnection.java  |    6 +-
 .../phoenix/jdbc/PhoenixDatabaseMetaData.java   |    3 +
 .../apache/phoenix/jdbc/PhoenixStatement.java   |    1 +
 .../phoenix/mapreduce/PhoenixRecordReader.java  |   11 +
 .../apache/phoenix/optimize/QueryOptimizer.java |    2 +-
 .../query/ConnectionQueryServicesImpl.java      |  104 +-
 .../query/ConnectionlessQueryServicesImpl.java  |    4 +-
 .../query/DelegateConnectionQueryServices.java  |    4 +-
 .../apache/phoenix/query/MetaDataMutated.java   |    2 +-
 .../apache/phoenix/query/QueryConstants.java    |    4 +-
 .../apache/phoenix/schema/DelegateTable.java    |    6 +
 .../apache/phoenix/schema/MetaDataClient.java   |  706 +++++-----
 .../org/apache/phoenix/schema/PMetaData.java    |    3 +-
 .../apache/phoenix/schema/PMetaDataImpl.java    |   15 +-
 .../java/org/apache/phoenix/schema/PTable.java  |    1 +
 .../org/apache/phoenix/schema/PTableImpl.java   |  104 +-
 .../org/apache/phoenix/schema/PTableRef.java    |   17 +-
 .../apache/phoenix/schema/TableProperty.java    |   26 +
 .../org/apache/phoenix/schema/TableRef.java     |   11 +-
 .../phoenix/schema/stats/GuidePostsInfo.java    |  106 +-
 .../schema/stats/GuidePostsInfoBuilder.java     |  113 ++
 .../phoenix/schema/stats/PTableStatsImpl.java   |   45 +-
 .../stats/StatisticsCollectionRunTracker.java   |   14 +-
 .../schema/stats/StatisticsCollector.java       |   46 +-
 .../phoenix/schema/stats/StatisticsScanner.java |    3 +-
 .../phoenix/schema/stats/StatisticsUtil.java    |   36 +-
 .../phoenix/schema/stats/StatisticsWriter.java  |   46 +-
 .../java/org/apache/phoenix/util/ByteUtil.java  |    2 +
 .../apache/phoenix/util/PrefixByteCodec.java    |  104 ++
 .../apache/phoenix/util/PrefixByteDecoder.java  |   90 ++
 .../apache/phoenix/util/PrefixByteEncoder.java  |   99 ++
 .../org/apache/phoenix/util/ReadOnlyProps.java  |   40 +-
 .../apache/phoenix/util/TransactionUtil.java    |   23 +-
 .../org/apache/phoenix/util/UpgradeUtil.java    |   68 +-
 .../phoenix/compile/QueryCompilerTest.java      |   14 +-
 .../phoenix/execute/CorrelatePlanTest.java      |    4 +-
 .../BuiltinFunctionConstructorTest.java         |   65 +
 .../phoenix/filter/SkipScanBigFilterTest.java   |    8 +-
 .../phoenix/filter/SkipScanFilterTest.java      |   65 +-
 .../phoenix/jdbc/ReadOnlyPropertiesTest.java    |   84 ++
 .../java/org/apache/phoenix/query/BaseTest.java |  103 +-
 .../phoenix/schema/PMetaDataImplTest.java       |  125 +-
 .../util/PrefixByteEncoderDecoderTest.java      |   96 ++
 .../java/org/apache/phoenix/util/TestUtil.java  |    2 +-
 .../apache/phoenix/pherf/PherfConstants.java    |    1 +
 .../phoenix/pherf/rules/RulesApplier.java       |    3 +-
 .../org/apache/phoenix/pherf/ResourceTest.java  |    2 +-
 .../apache/phoenix/pherf/RuleGeneratorTest.java |    7 +-
 phoenix-protocol/src/main/PGuidePosts.proto     |    3 +
 phoenix-protocol/src/main/PTable.proto          |    1 +
 pom.xml                                         |    7 +-
 101 files changed, 3367 insertions(+), 2853 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/beb8b813/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
index c1fc245,0fc6d74..999c299
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
@@@ -178,8 -178,8 +178,9 @@@ public class TupleProjectionCompiler 
                      null, table.getTimeStamp(), table.getSequenceNumber(), table.getPKName(),
                      retainPKColumns ? table.getBucketNum() : null, projectedColumns, null,
                      null, Collections.<PTable>emptyList(), table.isImmutableRows(), Collections.<PName>emptyList(), null, null,
 -                    table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(),
 -                    null, table.rowKeyOrderOptimizable(), table.isTransactional(), table.getUpdateCacheFrequency());
 +                    table.isWALDisabled(), retainPKColumns ? table.isMultiTenant() : false, table.getStoreNulls(), table.getViewType(),
-                     retainPKColumns ? table.getViewIndexId() : null, null, table.rowKeyOrderOptimizable(), table.isTransactional());
++                    retainPKColumns ? table.getViewIndexId() : null, null, table.rowKeyOrderOptimizable(), table.isTransactional(),
++                    table.getUpdateCacheFrequency());
      }
  
      // For extracting column references from single select statement

http://git-wip-us.apache.org/repos/asf/phoenix/blob/beb8b813/phoenix-core/src/main/java/org/apache/phoenix/execute/ScanPlan.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/beb8b813/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/beb8b813/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/beb8b813/pom.xml
----------------------------------------------------------------------


[50/50] [abbrv] phoenix git commit: Sync with master; Bug fix for ResultSet object of Float type; Walkaround for PHOENIX-2647

Posted by ma...@apache.org.
Sync with master; Bug fix for ResultSet object of Float type; Walkaround for PHOENIX-2647


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

Branch: refs/heads/calcite
Commit: 74409be82c1224b0cefee1b1e6a64bb3d0491339
Parents: beb8b81 6af7dcb
Author: maryannxue <ma...@gmail.com>
Authored: Mon Feb 1 22:59:16 2016 -0500
Committer: maryannxue <ma...@gmail.com>
Committed: Mon Feb 1 22:59:16 2016 -0500

----------------------------------------------------------------------
 phoenix-assembly/pom.xml                        |   2 +-
 phoenix-core/pom.xml                            |   2 +-
 .../apache/phoenix/calcite/BaseCalciteIT.java   |  91 ++++++++--
 .../org/apache/phoenix/calcite/CalciteIT.java   | 140 ++++++++--------
 .../apache/phoenix/calcite/CalciteIndexIT.java  |  36 ++--
 .../phoenix/end2end/CsvBulkLoadToolIT.java      |   5 +-
 .../apache/phoenix/end2end/DerivedTableIT.java  |  13 ++
 .../phoenix/end2end/StatsCollectorIT.java       |  73 ++++----
 .../phoenix/end2end/TransactionalViewIT.java    | 120 +++++++++++++
 .../phoenix/end2end/index/ImmutableIndexIT.java | 123 +++++++++-----
 .../end2end/index/MutableIndexFailureIT.java    |   6 +-
 .../apache/phoenix/execute/PartialCommitIT.java |   2 +-
 .../index/balancer/IndexLoadBalancerIT.java     |   2 +
 .../org/apache/phoenix/tx/TransactionIT.java    |  39 +----
 .../ipc/controller/MetadataRpcController.java   |   3 +-
 .../wal/BinaryCompatibleBaseDecoder.java        | 110 ++++++++++++
 .../regionserver/wal/IndexedWALEditCodec.java   |  83 ++++++++-
 .../cache/aggcache/SpillableGroupByCache.java   |  17 +-
 .../apache/phoenix/calcite/CalciteRuntime.java  |  10 +-
 .../apache/phoenix/calcite/PhoenixSchema.java   |   2 +-
 .../calcite/rel/PhoenixRelImplementorImpl.java  |   4 +-
 .../apache/phoenix/compile/FromCompiler.java    |   8 +-
 .../phoenix/compile/ListJarsQueryPlan.java      |   2 +-
 .../phoenix/compile/OrderPreservingTracker.java |   6 +-
 .../compile/PostLocalIndexDDLCompiler.java      | 104 ++++++++++++
 .../apache/phoenix/compile/TraceQueryPlan.java  |   2 +-
 .../apache/phoenix/compile/UnionCompiler.java   |   2 +-
 .../phoenix/coprocessor/BaseRegionScanner.java  |  11 +-
 .../GroupedAggregateRegionObserver.java         |  38 +----
 .../coprocessor/MetaDataEndpointImpl.java       |   7 +-
 .../coprocessor/MetaDataRegionObserver.java     |   8 +-
 .../phoenix/coprocessor/ScanRegionObserver.java |  17 +-
 .../UngroupedAggregateRegionObserver.java       |  24 +--
 .../coprocessor/generated/PTableProtos.java     | 145 +++++++++++++---
 .../phoenix/execute/ClientAggregatePlan.java    |   2 +-
 .../apache/phoenix/execute/HashJoinPlan.java    |   2 +-
 .../apache/phoenix/execute/MutationState.java   |  84 +++++-----
 .../index/write/KillServerOnFailurePolicy.java  |   5 +-
 .../index/PhoenixIndexFailurePolicy.java        |   9 +-
 .../phoenix/iterate/BaseResultIterators.java    |   2 +-
 .../apache/phoenix/jdbc/PhoenixStatement.java   |   1 -
 .../phoenix/mapreduce/AbstractBulkLoadTool.java |   2 +-
 .../mapreduce/PhoenixRecordWritable.java        |  28 ++--
 .../query/ConnectionQueryServicesImpl.java      |  82 ++-------
 .../org/apache/phoenix/query/QueryServices.java |   2 +
 .../phoenix/query/QueryServicesOptions.java     |  11 +-
 .../apache/phoenix/schema/DelegateColumn.java   |   5 +
 .../apache/phoenix/schema/MetaDataClient.java   | 133 ++++++---------
 .../java/org/apache/phoenix/schema/PColumn.java |   2 +
 .../org/apache/phoenix/schema/PColumnImpl.java  |  21 ++-
 .../apache/phoenix/schema/PMetaDataImpl.java    |   2 +-
 .../org/apache/phoenix/schema/SaltingUtil.java  |   2 +-
 .../phoenix/execute/CorrelatePlanTest.java      |   2 +-
 .../phoenix/execute/UnnestArrayPlanTest.java    |   4 +-
 .../expression/ColumnExpressionTest.java        |   8 +-
 .../phoenix/filter/SkipScanFilterTest.java      |  62 +++----
 .../iterate/AggregateResultScannerTest.java     |   4 +
 .../phoenix/query/QueryServicesTestImpl.java    |   4 +-
 phoenix-flume/pom.xml                           |   2 +-
 .../flume/serializer/RegexEventSerializer.java  |  12 +-
 phoenix-pherf/pom.xml                           |   2 +-
 phoenix-pig/pom.xml                             |   2 +-
 .../phoenix/pig/PhoenixHBaseLoaderIT.java       | 167 ++++++++++++++++++-
 .../phoenix/pig/PhoenixHBaseStorerIT.java       |  37 ++++
 .../apache/phoenix/pig/PhoenixHBaseLoader.java  |  22 +--
 .../apache/phoenix/pig/PhoenixHBaseStorage.java |  33 +++-
 .../org/apache/phoenix/pig/util/TypeUtil.java   |  49 ++++--
 .../pig/writable/PhoenixPigDBWritable.java      | 121 --------------
 .../apache/phoenix/pig/util/TypeUtilTest.java   |  39 +++--
 phoenix-protocol/src/main/MetaDataService.proto |   2 +-
 phoenix-protocol/src/main/PTable.proto          |   1 +
 phoenix-server-client/pom.xml                   |   2 +-
 phoenix-server/pom.xml                          |   2 +-
 phoenix-spark/pom.xml                           |   5 +-
 phoenix-tracing-webapp/pom.xml                  |   2 +-
 pom.xml                                         |   5 +-
 76 files changed, 1448 insertions(+), 793 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/74409be8/phoenix-core/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/74409be8/phoenix-core/src/it/java/org/apache/phoenix/calcite/BaseCalciteIT.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/it/java/org/apache/phoenix/calcite/BaseCalciteIT.java
index 55865f0,0000000..9afddab
mode 100644,000000..100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/calcite/BaseCalciteIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/calcite/BaseCalciteIT.java
@@@ -1,440 -1,0 +1,503 @@@
 +/*
 + * 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.phoenix.calcite;
 +
 +import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
 +import static org.junit.Assert.assertEquals;
 +import static org.junit.Assert.assertFalse;
 +import static org.junit.Assert.assertTrue;
 +import static org.junit.Assert.fail;
 +
 +import java.io.File;
 +import java.io.FileWriter;
 +import java.io.PrintWriter;
 +import java.sql.Connection;
 +import java.sql.DriverManager;
 +import java.sql.PreparedStatement;
 +import java.sql.ResultSet;
 +import java.sql.SQLException;
 +import java.sql.Statement;
 +import java.util.Arrays;
 +import java.util.List;
++import java.util.Map;
 +import java.util.Properties;
 +
 +import org.apache.calcite.avatica.util.ArrayImpl;
 +import org.apache.calcite.config.CalciteConnectionProperty;
 +import org.apache.phoenix.end2end.BaseClientManagedTimeIT;
++import org.apache.phoenix.query.QueryServices;
 +import org.apache.phoenix.schema.TableAlreadyExistsException;
 +import org.apache.phoenix.util.PhoenixRuntime;
 +import org.apache.phoenix.util.PropertiesUtil;
++import org.apache.phoenix.util.ReadOnlyProps;
++import org.junit.BeforeClass;
 +
 +import com.google.common.collect.Lists;
 +
 +public class BaseCalciteIT extends BaseClientManagedTimeIT {
 +    
++    @BeforeClass
++    public static void doSetup() throws Exception {
++        Map<String,String> props = getDefaultProps();
++        props.put(QueryServices.RUN_UPDATE_STATS_ASYNC, Boolean.FALSE.toString());
++        props.put(QueryServices.STATS_GUIDEPOST_WIDTH_BYTES_ATTRIB, Long.toString(1000));
++        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
++    }
++    
 +    public static Start start(boolean materializationEnabled) {
 +        return new Start(getConnectionProps(materializationEnabled));
 +    }
 +    
 +    public static Start start(Properties props) {
 +        return new Start(props);
 +    }
 +
 +    public static class Start {
 +        protected final Properties props;
 +        private Connection connection;
 +        
 +        Start(Properties props) {
 +            this.props = props;
 +        }
 +
 +        Connection createConnection() throws Exception {
 +            return DriverManager.getConnection(
 +                    "jdbc:phoenixcalcite:" 
 +                            + getUrl().substring(PhoenixRuntime.JDBC_PROTOCOL.length() + 1), 
 +                    props);
 +        }
 +
 +        public Sql sql(String sql) {
 +            return new Sql(this, sql);
 +        }
 +
 +        public Connection getConnection() {
 +            if (connection == null) {
 +                try {
 +                    connection = createConnection();
 +                } catch (Exception e) {
 +                    throw new RuntimeException(e);
 +                }
 +            }
 +            return connection;
 +        }
 +
 +        public void close() {
 +            if (connection != null) {
 +                try {
 +                    connection.close();
 +                } catch (SQLException e) {
 +                    throw new RuntimeException(e);
 +                }
 +            }
 +        }
 +    }
 +
 +    /** Fluid class for a test that has specified a SQL query. */
 +    static class Sql {
 +        private final Start start;
 +        private final String sql;
 +
 +        public Sql(Start start, String sql) {
 +            this.start = start;
 +            this.sql = sql;
 +        }
 +
 +        public static List<Object[]> getResult(ResultSet resultSet) throws SQLException {
 +            final List<Object[]> list = Lists.newArrayList();
 +            populateResult(resultSet, list);
 +            return list;
 +        }
 +
 +        private static void populateResult(ResultSet resultSet, List<Object[]> list) throws SQLException {
 +            final int columnCount = resultSet.getMetaData().getColumnCount();
 +            while (resultSet.next()) {
 +                Object[] row = new Object[columnCount];
 +                for (int i = 0; i < columnCount; i++) {
 +                    row[i] = resultSet.getObject(i + 1);
 +                }
 +                list.add(row);
 +            }
 +        }
 +
 +        public Sql explainIs(String expected) throws SQLException {
 +            final List<Object[]> list = getResult("explain plan for " + sql);
 +            if (list.size() != 1) {
 +                fail("explain should return 1 row, got " + list.size());
 +            }
 +            String explain = (String) (list.get(0)[0]);
 +            assertEquals(explain, expected);
 +            return this;
 +        }
 +
 +
 +        public boolean execute() throws SQLException {
 +            final Statement statement = start.getConnection().createStatement();
 +            final boolean execute = statement.execute(sql);
 +            statement.close();
 +            return execute;
 +        }
 +
 +        public List<Object[]> getResult(String sql) throws SQLException {
 +            final Statement statement = start.getConnection().createStatement();
 +            final ResultSet resultSet = statement.executeQuery(sql);
 +            List<Object[]> list = getResult(resultSet);
 +            resultSet.close();
 +            statement.close();
 +            return list;
 +        }
 +
 +        public void close() {
 +            start.close();
 +        }
 +
-         public Sql resultIs(Object[]... expected) throws SQLException {
++        public Sql resultIs(boolean ordered, Object[][] expected) throws SQLException {
 +            final Statement statement = start.getConnection().createStatement();
 +            final ResultSet resultSet = statement.executeQuery(sql);
-             for (int i = 0; i < expected.length; i++) {
-                 assertTrue(resultSet.next());
++            if (ordered) {
++                checkResultOrdered(resultSet, expected);
++            } else {
++                checkResultUnordered(resultSet, expected);
++            }
++            resultSet.close();
++            statement.close();
++            return this;
++        }
++        
++        private void checkResultOrdered(ResultSet resultSet, Object[][] expected) throws SQLException {
++            int expectedCount = expected.length;
++            int count = 0;
++            for (int i = 0; i < expectedCount; i++) {
++                assertTrue(
++                        "Expected " + expectedCount + " rows, but got " + count + " rows.",
++                        resultSet.next());
++                count++;
 +                Object[] row = expected[i];
 +                for (int j = 0; j < row.length; j++) {
 +                    Object obj = resultSet.getObject(j + 1);
-                     if (obj instanceof ArrayImpl) {
-                         assertEquals(
-                                 Arrays.toString((Object[]) row[j]),
-                                 obj.toString());
-                     } else {
-                         assertEquals(row[j], obj);
-                     }
++                    assertEquals(canonicalize(row[j]), canonicalize(obj));
 +                }
 +            }
-             assertFalse(resultSet.next());
-             resultSet.close();
-             statement.close();
-             return this;
++            assertFalse("Got more rows than expected.", resultSet.next());            
++        }
++        
++        private void checkResultUnordered(ResultSet resultSet, Object[][] expected) throws SQLException {
++            List<List<Object>> expectedResults = Lists.newArrayList();
++            List<List<Object>> actualResults = Lists.newArrayList();
++            List<List<Object>> errorResults = Lists.newArrayList();
++            int columnCount = expected.length > 0 ? expected[0].length : 0;
++            for (Object[] e : expected) {
++                List<Object> row = Lists.newArrayList();
++                for (Object o : e) {
++                    row.add(canonicalize(o));
++                }
++                expectedResults.add(row);
++            }
++            while (resultSet.next()) {
++                List<Object> result = Lists.newArrayList();
++                for (int i = 0; i < columnCount; i++) {
++                    result.add(canonicalize(resultSet.getObject(i+1)));
++                }
++                if (!expectedResults.remove(result)) {
++                    errorResults.add(result);
++                }
++                actualResults.add(result);
++            }
++            assertTrue(
++                    (expectedResults.isEmpty() ? "" : ("Count not find " + expectedResults + " in actual results: " + actualResults + ".\n")) +
++                    (errorResults.isEmpty() ? "" : "Could not find " + errorResults + " in expected results.\n"),
++                    errorResults.isEmpty() && expectedResults.isEmpty());
++        }
++        
++        private Object canonicalize(Object obj) {
++            if (obj == null) {
++                return obj;
++            }
++            
++            if (obj instanceof ArrayImpl) {
++                return obj.toString();
++            }
++            
++            if (obj.getClass().isArray()) {
++                return Arrays.toString((Object[]) obj);
++            }
++            
++            return obj;
 +        }
 +    }
 +
 +    private static final String FOODMART_SCHEMA = "     {\n"
 +            + "       type: 'jdbc',\n"
 +            + "       name: 'foodmart',\n"
 +            + "       jdbcDriver: 'org.hsqldb.jdbcDriver',\n"
 +            + "       jdbcUser: 'FOODMART',\n"
 +            + "       jdbcPassword: 'FOODMART',\n"
 +            + "       jdbcUrl: 'jdbc:hsqldb:res:foodmart',\n"
 +            + "       jdbcCatalog: null,\n"
 +            + "       jdbcSchema: 'foodmart'\n"
 +            + "     }";
 +    
 +    private static final String getPhoenixSchema() {
 +        return "    {\n"
 +            + "      name: 'phoenix',\n"
 +            + "      type: 'custom',\n"
 +            + "      factory: 'org.apache.phoenix.calcite.PhoenixSchema$Factory',\n"
 +            + "      operand: {\n"
 +            + "        url: \"" + getUrl() + "\"\n"
 +            + "      }\n"
 +            + "    }";
 +    }
 +
 +    protected static Connection connectUsingModel(Properties props) throws Exception {
 +        final File file = File.createTempFile("model", ".json");
 +        final String url = getUrl();
 +        final PrintWriter pw = new PrintWriter(new FileWriter(file));
 +        pw.print(
 +            "{\n"
 +                + "  version: '1.0',\n"
 +                + "  defaultSchema: 'HR',\n"
 +                + "  schemas: [\n"
 +                + "    {\n"
 +                + "      name: 'HR',\n"
 +                + "      type: 'custom',\n"
 +                + "      factory: 'org.apache.phoenix.calcite.PhoenixSchema$Factory',\n"
 +                + "      operand: {\n"
 +                + "        url: \"" + url + "\",\n"
 +                + "        user: \"scott\",\n"
 +                + "        password: \"tiger\"\n"
 +                + "      }\n"
 +                + "    }\n"
 +                + "  ]\n"
 +                + "}\n");
 +        pw.close();
 +        final Connection connection =
 +            DriverManager.getConnection("jdbc:phoenixcalcite:model=" + file.getAbsolutePath(), props);
 +        return connection;
 +    }
 +
 +    protected static Connection connectWithHsqldbUsingModel(Properties props) throws Exception {
 +        final File file = File.createTempFile("model", ".json");
 +        final PrintWriter pw = new PrintWriter(new FileWriter(file));
 +        pw.print(
 +            "{\n"
 +                + "  version: '1.0',\n"
 +                + "  defaultSchema: 'phoenix',\n"
 +                + "  schemas: [\n"
 +                + getPhoenixSchema() + ",\n"
 +                + FOODMART_SCHEMA + "\n"
 +                + "  ]\n"
 +                + "}\n");
 +        pw.close();
 +        final Connection connection =
 +            DriverManager.getConnection("jdbc:phoenixcalcite:model=" + file.getAbsolutePath(), props);
 +        return connection;
 +    }
 +
 +    protected static Properties getConnectionProps(boolean enableMaterialization) {
 +        Properties props = new Properties();
 +        props.setProperty(
 +                CalciteConnectionProperty.MATERIALIZATIONS_ENABLED.camelName(),
 +                Boolean.toString(enableMaterialization));
 +        props.setProperty(
 +                CalciteConnectionProperty.CREATE_MATERIALIZATIONS.camelName(),
 +                Boolean.toString(false));
 +        return props;
 +    }
 +    
 +    protected static final String SCORES_TABLE_NAME = "scores";
 +    
 +    protected void initArrayTable() throws Exception {
 +        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
 +        Connection conn = DriverManager.getConnection(getUrl(), props);
 +        try {
 +            conn.createStatement().execute(
 +                    "CREATE TABLE " + SCORES_TABLE_NAME
 +                    + "(student_id INTEGER NOT NULL, subject_id INTEGER NOT NULL, scores INTEGER[] CONSTRAINT pk PRIMARY KEY (student_id, subject_id))");
 +            PreparedStatement stmt = conn.prepareStatement(
 +                    "UPSERT INTO " + SCORES_TABLE_NAME
 +                    + " VALUES(?, ?, ?)");
 +            stmt.setInt(1, 1);
 +            stmt.setInt(2, 1);
 +            stmt.setArray(3, conn.createArrayOf("INTEGER", new Integer[] {85, 80, 82}));
 +            stmt.execute();
 +            stmt.setInt(1, 2);
 +            stmt.setInt(2, 1);
 +            stmt.setArray(3, null);
 +            stmt.execute();
 +            stmt.setInt(1, 3);
 +            stmt.setInt(2, 2);
 +            stmt.setArray(3, conn.createArrayOf("INTEGER", new Integer[] {87, 88, 80}));
 +            stmt.execute();
 +            conn.commit();
 +        } catch (TableAlreadyExistsException e) {
 +        }
 +        conn.close();        
 +    }
 +    
 +    protected static final String NOSALT_TABLE_NAME = "nosalt_test_table";
 +    protected static final String NOSALT_TABLE_SALTED_INDEX_NAME = "idxsalted_nosalt_test_table";
 +    protected static final String SALTED_TABLE_NAME = "salted_test_table";
 +    protected static final String SALTED_TABLE_NOSALT_INDEX_NAME = "idx_salted_test_table";
 +    protected static final String SALTED_TABLE_SALTED_INDEX_NAME = "idxsalted_salted_test_table";
 +    
 +    protected void initSaltedTables(String index) throws SQLException {
 +        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
 +        Connection conn = DriverManager.getConnection(getUrl(), props);
 +        try {
 +            conn.createStatement().execute(
 +                    "CREATE TABLE " + NOSALT_TABLE_NAME + " (mypk0 INTEGER NOT NULL, mypk1 INTEGER NOT NULL, col0 INTEGER, col1 INTEGER CONSTRAINT pk PRIMARY KEY (mypk0, mypk1))");
 +            PreparedStatement stmt = conn.prepareStatement(
 +                    "UPSERT INTO " + NOSALT_TABLE_NAME
 +                    + " VALUES(?, ?, ?, ?)");
 +            stmt.setInt(1, 1);
 +            stmt.setInt(2, 2);
 +            stmt.setInt(3, 3);
 +            stmt.setInt(4, 4);
 +            stmt.execute();
 +            stmt.setInt(1, 2);
 +            stmt.setInt(2, 3);
 +            stmt.setInt(3, 4);
 +            stmt.setInt(4, 5);
 +            stmt.execute();
 +            stmt.setInt(1, 3);
 +            stmt.setInt(2, 4);
 +            stmt.setInt(3, 5);
 +            stmt.setInt(4, 6);
 +            stmt.execute();
 +            conn.commit();
 +            
 +            if (index != null) {
 +                conn.createStatement().execute("CREATE " + index + " " + NOSALT_TABLE_SALTED_INDEX_NAME + " ON " + NOSALT_TABLE_NAME + " (col0) SALT_BUCKETS=4");
 +                conn.commit();
 +            }
 +            
 +            conn.createStatement().execute(
 +                    "CREATE TABLE " + SALTED_TABLE_NAME + " (mypk0 INTEGER NOT NULL, mypk1 INTEGER NOT NULL, col0 INTEGER, col1 INTEGER CONSTRAINT pk PRIMARY KEY (mypk0, mypk1)) SALT_BUCKETS=4");
 +            stmt = conn.prepareStatement(
 +                    "UPSERT INTO " + SALTED_TABLE_NAME
 +                    + " VALUES(?, ?, ?, ?)");
 +            stmt.setInt(1, 1);
 +            stmt.setInt(2, 2);
 +            stmt.setInt(3, 3);
 +            stmt.setInt(4, 4);
 +            stmt.execute();
 +            stmt.setInt(1, 2);
 +            stmt.setInt(2, 3);
 +            stmt.setInt(3, 4);
 +            stmt.setInt(4, 5);
 +            stmt.execute();
 +            stmt.setInt(1, 3);
 +            stmt.setInt(2, 4);
 +            stmt.setInt(3, 5);
 +            stmt.setInt(4, 6);
 +            stmt.execute();
 +            conn.commit();
 +            
 +            if (index != null) {
 +                conn.createStatement().execute("CREATE " + index + " " + SALTED_TABLE_NOSALT_INDEX_NAME + " ON " + SALTED_TABLE_NAME + " (col0)");
 +                conn.createStatement().execute("CREATE " + index + " " + SALTED_TABLE_SALTED_INDEX_NAME + " ON " + SALTED_TABLE_NAME + " (col1) INCLUDE (col0) SALT_BUCKETS=4");
 +                conn.commit();
 +            }
 +        } catch (TableAlreadyExistsException e) {
 +        }
 +        conn.close();        
 +    }
 +    
 +    protected static final String MULTI_TENANT_TABLE = "multitenant_test_table";
 +    protected static final String MULTI_TENANT_TABLE_INDEX = "idx_multitenant_test_table";
 +    protected static final String MULTI_TENANT_VIEW1 = "s1.multitenant_test_view1";
 +    protected static final String MULTI_TENANT_VIEW1_INDEX = "idx_multitenant_test_view1";
 +    protected static final String MULTI_TENANT_VIEW2 = "s2.multitenant_test_view2";
 +    protected static final String MULTI_TENANT_VIEW2_INDEX = "idx_multitenant_test_view2";
 +    
 +    protected void initMultiTenantTables(String index) throws SQLException {
 +        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
 +        Connection conn = DriverManager.getConnection(getUrl(), props);
 +        try {
 +            conn.createStatement().execute(
 +                    "CREATE TABLE " + MULTI_TENANT_TABLE + " (tenant_id VARCHAR NOT NULL, id VARCHAR NOT NULL, col0 INTEGER, col1 INTEGER, col2 INTEGER CONSTRAINT pk PRIMARY KEY (tenant_id, id)) MULTI_TENANT=true");
 +            PreparedStatement stmt = conn.prepareStatement(
 +                    "UPSERT INTO " + MULTI_TENANT_TABLE
 +                    + " VALUES(?, ?, ?, ?, ?)");
 +            stmt.setString(1, "10");
 +            stmt.setString(2, "2");
 +            stmt.setInt(3, 3);
 +            stmt.setInt(4, 4);
 +            stmt.setInt(5, 5);
 +            stmt.execute();
 +            stmt.setString(1, "15");
 +            stmt.setString(2, "3");
 +            stmt.setInt(3, 4);
 +            stmt.setInt(4, 5);
 +            stmt.setInt(5, 6);
 +            stmt.execute();
 +            stmt.setString(1, "20");
 +            stmt.setString(2, "4");
 +            stmt.setInt(3, 5);
 +            stmt.setInt(4, 6);
 +            stmt.setInt(5, 7);
 +            stmt.execute();
 +            stmt.setString(1, "20");
 +            stmt.setString(2, "5");
 +            stmt.setInt(3, 6);
 +            stmt.setInt(4, 7);
 +            stmt.setInt(5, 8);
 +            stmt.execute();
 +            conn.commit();
 +            
 +            if (index != null) {
 +                conn.createStatement().execute(
 +                        "CREATE " + index + " " + MULTI_TENANT_TABLE_INDEX
 +                        + " ON " + MULTI_TENANT_TABLE + "(col1) INCLUDE (col0, col2)");
 +                conn.commit();
 +            }
 +            
 +            conn.close();
 +            props.setProperty("TenantId", "10");
 +            conn = DriverManager.getConnection(getUrl(), props);
 +            conn.createStatement().execute("CREATE VIEW " + MULTI_TENANT_VIEW1
 +                    + " AS select * from " + MULTI_TENANT_TABLE);
 +            conn.commit();
 +            
 +            if (index != null) {
 +                conn.createStatement().execute(
 +                        "CREATE " + index + " " + MULTI_TENANT_VIEW1_INDEX
 +                        + " ON " + MULTI_TENANT_VIEW1 + "(col0)");
 +                conn.commit();
 +            }
 +            
 +            conn.close();
 +            props.setProperty("TenantId", "20");
 +            conn = DriverManager.getConnection(getUrl(), props);
 +            conn.createStatement().execute("CREATE VIEW " + MULTI_TENANT_VIEW2
 +                    + " AS select * from " + MULTI_TENANT_TABLE + " where col2 > 7");
 +            conn.commit();
 +            
 +            if (index != null) {
 +                conn.createStatement().execute(
 +                        "CREATE " + index + " " + MULTI_TENANT_VIEW2_INDEX
 +                        + " ON " + MULTI_TENANT_VIEW2 + "(col0)");
 +                conn.commit();
 +            }
 +        } catch (TableAlreadyExistsException e) {
 +        } finally {
 +            conn.close();
 +        }
 +    }
 +
 +}


[17/50] [abbrv] phoenix git commit: PHOENIX-2589 Fix a few resource leaks, NULL dereference, NULL_RETURNS issues (Samarth Jain, Alicia Ying Shu)

Posted by ma...@apache.org.
PHOENIX-2589 Fix a few resource leaks, NULL dereference, NULL_RETURNS issues (Samarth Jain, Alicia Ying Shu)


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

Branch: refs/heads/calcite
Commit: 6911770e8a1d05775f3780f623ae01e9122d59f0
Parents: 6ecbbb2
Author: Samarth <sa...@salesforce.com>
Authored: Wed Jan 20 17:07:12 2016 -0800
Committer: Samarth <sa...@salesforce.com>
Committed: Wed Jan 20 17:07:12 2016 -0800

----------------------------------------------------------------------
 .../query/ConnectionQueryServicesImpl.java      |  20 +-
 .../apache/phoenix/schema/MetaDataClient.java   | 235 ++++++++++---------
 pom.xml                                         |   1 +
 3 files changed, 137 insertions(+), 119 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/6911770e/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 a246e63..4522cf8 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
@@ -3311,8 +3311,14 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         wait = false;
                     }
                     // It is guaranteed that this poll won't hang indefinitely because this is the
-                    // only thread that removes items from the queue.
-                    WeakReference<PhoenixConnection> connRef = connectionsQueue.poll();
+                    // only thread that removes items from the queue. Still adding a 1 ms timeout
+                    // for sanity check.
+                    WeakReference<PhoenixConnection> connRef =
+                            connectionsQueue.poll(1, TimeUnit.MILLISECONDS);
+                    if (connRef == null) {
+                        throw new IllegalStateException(
+                                "Connection ref found to be null. This is a bug. Some other thread removed items from the connection queue.");
+                    }
                     PhoenixConnection conn = connRef.get();
                     if (conn != null && !conn.isClosed()) {
                         LinkedBlockingQueue<WeakReference<TableResultIterator>> scannerQueue =
@@ -3323,7 +3329,15 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         int renewed = 0;
                         long start = System.currentTimeMillis();
                         while (numScanners > 0) {
-                            WeakReference<TableResultIterator> ref = scannerQueue.poll();
+                            // It is guaranteed that this poll won't hang indefinitely because this is the
+                            // only thread that removes items from the queue. Still adding a 1 ms timeout
+                            // for sanity check.
+                            WeakReference<TableResultIterator> ref =
+                                    scannerQueue.poll(1, TimeUnit.MILLISECONDS);
+                            if (ref == null) {
+                                throw new IllegalStateException(
+                                        "TableResulIterator ref found to be null. This is a bug. Some other thread removed items from the scanner queue.");
+                            }
                             TableResultIterator scanningItr = ref.get();
                             if (scanningItr != null) {
                                 RenewLeaseStatus status = scanningItr.renewLease();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6911770e/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 0b446b3..064007f 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
@@ -1450,25 +1450,26 @@ public class MetaDataClient {
             List<Mutation> functionData = Lists.newArrayListWithExpectedSize(function.getFunctionArguments().size() + 1);
 
             List<FunctionArgument> args = function.getFunctionArguments();
-            PreparedStatement argUpsert = connection.prepareStatement(INSERT_FUNCTION_ARGUMENT);
-
-            for (int i = 0; i < args.size(); i++) {
-                FunctionArgument arg = args.get(i);
-                addFunctionArgMutation(function.getFunctionName(), arg, argUpsert, i);
+            try (PreparedStatement argUpsert = connection.prepareStatement(INSERT_FUNCTION_ARGUMENT)) {
+                for (int i = 0; i < args.size(); i++) {
+                    FunctionArgument arg = args.get(i);
+                    addFunctionArgMutation(function.getFunctionName(), arg, argUpsert, i);
+                }
+                functionData.addAll(connection.getMutationState().toMutations().next().getSecond());
+                connection.rollback();
             }
-            functionData.addAll(connection.getMutationState().toMutations().next().getSecond());
-            connection.rollback();
 
-            PreparedStatement functionUpsert = connection.prepareStatement(CREATE_FUNCTION);
-            functionUpsert.setString(1, tenantIdStr);
-            functionUpsert.setString(2, function.getFunctionName());
-            functionUpsert.setInt(3, function.getFunctionArguments().size());
-            functionUpsert.setString(4, function.getClassName());
-            functionUpsert.setString(5, function.getJarPath());
-            functionUpsert.setString(6, function.getReturnType());
-            functionUpsert.execute();
-            functionData.addAll(connection.getMutationState().toMutations(null).next().getSecond());
-            connection.rollback();
+            try (PreparedStatement functionUpsert = connection.prepareStatement(CREATE_FUNCTION)) {
+                functionUpsert.setString(1, tenantIdStr);
+                functionUpsert.setString(2, function.getFunctionName());
+                functionUpsert.setInt(3, function.getFunctionArguments().size());
+                functionUpsert.setString(4, function.getClassName());
+                functionUpsert.setString(5, function.getJarPath());
+                functionUpsert.setString(6, function.getReturnType());
+                functionUpsert.execute();
+                functionData.addAll(connection.getMutationState().toMutations(null).next().getSecond());
+                connection.rollback();
+            }
             MetaDataMutationResult result = connection.getQueryServices().createFunction(functionData, function, stmt.isTemporary());
             MutationCode code = result.getMutationCode();
             switch(code) {
@@ -1880,7 +1881,6 @@ public class MetaDataClient {
                 }
             }
 
-            PreparedStatement colUpsert = connection.prepareStatement(INSERT_COLUMN_CREATE_TABLE);
             Map<String, PName> familyNames = Maps.newLinkedHashMap();
             boolean isPK = false;
             boolean rowTimeStampColumnAlreadyFound = false;
@@ -2056,38 +2056,40 @@ public class MetaDataClient {
             }
 
             short nextKeySeq = 0;
-            for (int i = 0; i < columns.size(); i++) {
-                PColumn column = columns.get(i);
-                final int columnPosition = column.getPosition();
-                // For client-side cache, we need to update the column
-                if (isViewColumnReferenced != null) {
-                    if (viewColumnConstants != null && columnPosition < viewColumnConstants.length) {
-                        columns.set(i, column = new DelegateColumn(column) {
-                            @Override
-                            public byte[] getViewConstant() {
-                                return viewColumnConstants[columnPosition];
-                            }
-                            @Override
-                            public boolean isViewReferenced() {
-                                return isViewColumnReferenced.get(columnPosition);
-                            }
-                        });
-                    } else {
-                        columns.set(i, column = new DelegateColumn(column) {
-                            @Override
-                            public boolean isViewReferenced() {
-                                return isViewColumnReferenced.get(columnPosition);
-                            }
-                        });
+            
+            try (PreparedStatement colUpsert = connection.prepareStatement(INSERT_COLUMN_CREATE_TABLE)) {
+                for (int i = 0; i < columns.size(); i++) {
+                    PColumn column = columns.get(i);
+                    final int columnPosition = column.getPosition();
+                    // For client-side cache, we need to update the column
+                    if (isViewColumnReferenced != null) {
+                        if (viewColumnConstants != null && columnPosition < viewColumnConstants.length) {
+                            columns.set(i, column = new DelegateColumn(column) {
+                                @Override
+                                public byte[] getViewConstant() {
+                                    return viewColumnConstants[columnPosition];
+                                }
+                                @Override
+                                public boolean isViewReferenced() {
+                                    return isViewColumnReferenced.get(columnPosition);
+                                }
+                            });
+                        } else {
+                            columns.set(i, column = new DelegateColumn(column) {
+                                @Override
+                                public boolean isViewReferenced() {
+                                    return isViewColumnReferenced.get(columnPosition);
+                                }
+                            });
+                        }
                     }
+                    Short keySeq = SchemaUtil.isPKColumn(column) ? ++nextKeySeq : null;
+                    addColumnMutation(schemaName, tableName, column, colUpsert, parentTableName, pkName, keySeq, saltBucketNum != null);
                 }
-                Short keySeq = SchemaUtil.isPKColumn(column) ? ++nextKeySeq : null;
-                addColumnMutation(schemaName, tableName, column, colUpsert, parentTableName, pkName, keySeq, saltBucketNum != null);
+                tableMetaData.addAll(connection.getMutationState().toMutations(timestamp).next().getSecond());
+                connection.rollback();
             }
 
-            tableMetaData.addAll(connection.getMutationState().toMutations(timestamp).next().getSecond());
-            connection.rollback();
-
             String dataTableName = parent == null || tableType == PTableType.VIEW ? null : parent.getTableName().getString();
             PIndexState indexState = parent == null || tableType == PTableType.VIEW  ? null : PIndexState.BUILDING;
             PreparedStatement tableUpsert = connection.prepareStatement(CREATE_TABLE);
@@ -2572,12 +2574,13 @@ public class MetaDataClient {
                         TABLE_NAME + "," +
                         propertyName +
                         ") VALUES (?, ?, ?, ?)";
-        PreparedStatement tableBoolUpsert = connection.prepareStatement(updatePropertySql);
-        tableBoolUpsert.setString(1, tenantId);
-        tableBoolUpsert.setString(2, schemaName);
-        tableBoolUpsert.setString(3, tableName);
-        tableBoolUpsert.setBoolean(4, propertyValue);
-        tableBoolUpsert.execute();
+        try (PreparedStatement tableBoolUpsert = connection.prepareStatement(updatePropertySql)) {
+            tableBoolUpsert.setString(1, tenantId);
+            tableBoolUpsert.setString(2, schemaName);
+            tableBoolUpsert.setString(3, tableName);
+            tableBoolUpsert.setBoolean(4, propertyValue);
+            tableBoolUpsert.execute();
+        }
     }
 
     private void mutateLongProperty(String tenantId, String schemaName, String tableName,
@@ -2588,12 +2591,13 @@ public class MetaDataClient {
                         TABLE_NAME + "," +
                         propertyName +
                         ") VALUES (?, ?, ?, ?)";
-        PreparedStatement tableBoolUpsert = connection.prepareStatement(updatePropertySql);
-        tableBoolUpsert.setString(1, tenantId);
-        tableBoolUpsert.setString(2, schemaName);
-        tableBoolUpsert.setString(3, tableName);
-        tableBoolUpsert.setLong(4, propertyValue);
-        tableBoolUpsert.execute();
+        try (PreparedStatement tableBoolUpsert = connection.prepareStatement(updatePropertySql)) {
+            tableBoolUpsert.setString(1, tenantId);
+            tableBoolUpsert.setString(2, schemaName);
+            tableBoolUpsert.setString(3, tableName);
+            tableBoolUpsert.setLong(4, propertyValue);
+            tableBoolUpsert.execute();
+        }
     }
 
     public MutationState addColumn(AddColumnStatement statement) throws SQLException {
@@ -2743,77 +2747,76 @@ public class MetaDataClient {
                 Long timeStamp = TransactionUtil.getTableTimestamp(connection, table.isTransactional() || nonTxToTx);
 
                 int numPkColumnsAdded = 0;
-                PreparedStatement colUpsert = connection.prepareStatement(INSERT_COLUMN_ALTER_TABLE);
-
                 List<PColumn> columns = Lists.newArrayListWithExpectedSize(columnDefs.size());
                 Set<String> colFamiliesForPColumnsToBeAdded = new LinkedHashSet<>();
                 Set<String> families = new LinkedHashSet<>();
                 if (columnDefs.size() > 0 ) {
-                    short nextKeySeq = SchemaUtil.getMaxKeySeq(table);
-                    for( ColumnDef colDef : columnDefs) {
-                        if (colDef != null && !colDef.isNull()) {
-                            if(colDef.isPK()) {
-                                throw new SQLExceptionInfo.Builder(SQLExceptionCode.NOT_NULLABLE_COLUMN_IN_ROW_KEY)
-                                .setColumnName(colDef.getColumnDefName().getColumnName()).build().buildException();
-                            } else {
-                                throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ADD_NOT_NULLABLE_COLUMN)
+                    try (PreparedStatement colUpsert = connection.prepareStatement(INSERT_COLUMN_ALTER_TABLE)) {
+                        short nextKeySeq = SchemaUtil.getMaxKeySeq(table);
+                        for( ColumnDef colDef : columnDefs) {
+                            if (colDef != null && !colDef.isNull()) {
+                                if(colDef.isPK()) {
+                                    throw new SQLExceptionInfo.Builder(SQLExceptionCode.NOT_NULLABLE_COLUMN_IN_ROW_KEY)
+                                    .setColumnName(colDef.getColumnDefName().getColumnName()).build().buildException();
+                                } else {
+                                    throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ADD_NOT_NULLABLE_COLUMN)
+                                    .setColumnName(colDef.getColumnDefName().getColumnName()).build().buildException();
+                                }
+                            }
+                            if (colDef != null && colDef.isPK() && table.getType() == VIEW && table.getViewType() != MAPPED) {
+                                throwIfLastPKOfParentIsFixedLength(getParentOfView(table), schemaName, tableName, colDef);
+                            }
+                            if (colDef != null && colDef.isRowTimestamp()) {
+                                throw new SQLExceptionInfo.Builder(SQLExceptionCode.ROWTIMESTAMP_CREATE_ONLY)
                                 .setColumnName(colDef.getColumnDefName().getColumnName()).build().buildException();
                             }
-                        }
-                        if (colDef != null && colDef.isPK() && table.getType() == VIEW && table.getViewType() != MAPPED) {
-                            throwIfLastPKOfParentIsFixedLength(getParentOfView(table), schemaName, tableName, colDef);
-                        }
-                        if (colDef != null && colDef.isRowTimestamp()) {
-                            throw new SQLExceptionInfo.Builder(SQLExceptionCode.ROWTIMESTAMP_CREATE_ONLY)
-                            .setColumnName(colDef.getColumnDefName().getColumnName()).build().buildException();
-                        }
-                        PColumn column = newColumn(position++, colDef, PrimaryKeyConstraint.EMPTY, table.getDefaultFamilyName() == null ? null : table.getDefaultFamilyName().getString(), true);
-                        columns.add(column);
-                        String pkName = null;
-                        Short keySeq = null;
-
-                        // TODO: support setting properties on other families?
-                        if (column.getFamilyName() == null) {
-                            ++numPkColumnsAdded;
-                            pkName = table.getPKName() == null ? null : table.getPKName().getString();
-                            keySeq = ++nextKeySeq;
-                        } else {
-                            families.add(column.getFamilyName().getString());
-                        }
-                        colFamiliesForPColumnsToBeAdded.add(column.getFamilyName() == null ? null : column.getFamilyName().getString());
-                        addColumnMutation(schemaName, tableName, column, colUpsert, null, pkName, keySeq, table.getBucketNum() != null);
-                    }
-
-                    // Add any new PK columns to end of index PK
-                    if (numPkColumnsAdded>0) {
-                        // create PK column list that includes the newly created columns
-                        List<PColumn> pkColumns = Lists.newArrayListWithExpectedSize(table.getPKColumns().size()+numPkColumnsAdded);
-                        pkColumns.addAll(table.getPKColumns());
-                        for (int i=0; i<columnDefs.size(); ++i) {
-                            if (columnDefs.get(i).isPK()) {
-                                pkColumns.add(columns.get(i));
+                            PColumn column = newColumn(position++, colDef, PrimaryKeyConstraint.EMPTY, table.getDefaultFamilyName() == null ? null : table.getDefaultFamilyName().getString(), true);
+                            columns.add(column);
+                            String pkName = null;
+                            Short keySeq = null;
+
+                            // TODO: support setting properties on other families?
+                            if (column.getFamilyName() == null) {
+                                ++numPkColumnsAdded;
+                                pkName = table.getPKName() == null ? null : table.getPKName().getString();
+                                keySeq = ++nextKeySeq;
+                            } else {
+                                families.add(column.getFamilyName().getString());
                             }
+                            colFamiliesForPColumnsToBeAdded.add(column.getFamilyName() == null ? null : column.getFamilyName().getString());
+                            addColumnMutation(schemaName, tableName, column, colUpsert, null, pkName, keySeq, table.getBucketNum() != null);
                         }
-                        int pkSlotPosition = table.getPKColumns().size()-1;
-                        for (PTable index : table.getIndexes()) {
-                            short nextIndexKeySeq = SchemaUtil.getMaxKeySeq(index);
-                            int indexPosition = index.getColumns().size();
+
+                        // Add any new PK columns to end of index PK
+                        if (numPkColumnsAdded>0) {
+                            // create PK column list that includes the newly created columns
+                            List<PColumn> pkColumns = Lists.newArrayListWithExpectedSize(table.getPKColumns().size()+numPkColumnsAdded);
+                            pkColumns.addAll(table.getPKColumns());
                             for (int i=0; i<columnDefs.size(); ++i) {
-                                ColumnDef colDef = columnDefs.get(i);
-                                if (colDef.isPK()) {
-                                    PDataType indexColDataType = IndexUtil.getIndexColumnDataType(colDef.isNull(), colDef.getDataType());
-                                    ColumnName indexColName = ColumnName.caseSensitiveColumnName(IndexUtil.getIndexColumnName(null, colDef.getColumnDefName().getColumnName()));
-                                    Expression expression = new RowKeyColumnExpression(columns.get(i), new RowKeyValueAccessor(pkColumns, ++pkSlotPosition));
-                                    ColumnDef indexColDef = FACTORY.columnDef(indexColName, indexColDataType.getSqlTypeName(), colDef.isNull(), colDef.getMaxLength(), colDef.getScale(), true, colDef.getSortOrder(), expression.toString(), colDef.isRowTimestamp());
-                                    PColumn indexColumn = newColumn(indexPosition++, indexColDef, PrimaryKeyConstraint.EMPTY, null, true);
-                                    addColumnMutation(schemaName, index.getTableName().getString(), indexColumn, colUpsert, index.getParentTableName().getString(), index.getPKName() == null ? null : index.getPKName().getString(), ++nextIndexKeySeq, index.getBucketNum() != null);
+                                if (columnDefs.get(i).isPK()) {
+                                    pkColumns.add(columns.get(i));
+                                }
+                            }
+                            int pkSlotPosition = table.getPKColumns().size()-1;
+                            for (PTable index : table.getIndexes()) {
+                                short nextIndexKeySeq = SchemaUtil.getMaxKeySeq(index);
+                                int indexPosition = index.getColumns().size();
+                                for (int i=0; i<columnDefs.size(); ++i) {
+                                    ColumnDef colDef = columnDefs.get(i);
+                                    if (colDef.isPK()) {
+                                        PDataType indexColDataType = IndexUtil.getIndexColumnDataType(colDef.isNull(), colDef.getDataType());
+                                        ColumnName indexColName = ColumnName.caseSensitiveColumnName(IndexUtil.getIndexColumnName(null, colDef.getColumnDefName().getColumnName()));
+                                        Expression expression = new RowKeyColumnExpression(columns.get(i), new RowKeyValueAccessor(pkColumns, ++pkSlotPosition));
+                                        ColumnDef indexColDef = FACTORY.columnDef(indexColName, indexColDataType.getSqlTypeName(), colDef.isNull(), colDef.getMaxLength(), colDef.getScale(), true, colDef.getSortOrder(), expression.toString(), colDef.isRowTimestamp());
+                                        PColumn indexColumn = newColumn(indexPosition++, indexColDef, PrimaryKeyConstraint.EMPTY, null, true);
+                                        addColumnMutation(schemaName, index.getTableName().getString(), indexColumn, colUpsert, index.getParentTableName().getString(), index.getPKName() == null ? null : index.getPKName().getString(), ++nextIndexKeySeq, index.getBucketNum() != null);
+                                    }
                                 }
                             }
                         }
+                        columnMetaData.addAll(connection.getMutationState().toMutations(timeStamp).next().getSecond());
+                        connection.rollback();
                     }
-
-                    columnMetaData.addAll(connection.getMutationState().toMutations(timeStamp).next().getSecond());
-                    connection.rollback();
                 } else {
                     // Check that HBase configured properly for mutable secondary indexing
                     // if we're changing from an immutable table to a mutable table and we

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6911770e/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 7191b1e..e635ae7 100644
--- a/pom.xml
+++ b/pom.xml
@@ -406,6 +406,7 @@
           <argLine>-enableassertions -Xmx2250m -XX:MaxPermSize=128m 
             -Djava.security.egd=file:/dev/./urandom "-Djava.library.path=${hadoop.library.path}${path.separator}${java.library.path}"</argLine>
           <redirectTestOutputToFile>${test.output.tofile}</redirectTestOutputToFile>
+          <shutdown>kill</shutdown>
         </configuration>
       </plugin>
       <!-- All projects create a test jar -->


[29/50] [abbrv] phoenix git commit: PHOENIX-2613 Uncomment test

Posted by ma...@apache.org.
PHOENIX-2613 Uncomment test


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

Branch: refs/heads/calcite
Commit: 5d61fb818e9dedef00b8d511cde7c3aeca490136
Parents: 9e90fab
Author: Samarth <sa...@salesforce.com>
Authored: Fri Jan 22 12:22:53 2016 -0800
Committer: Samarth <sa...@salesforce.com>
Committed: Fri Jan 22 12:22:53 2016 -0800

----------------------------------------------------------------------
 .../phoenix/filter/SkipScanFilterTest.java      | 62 ++++++++++----------
 1 file changed, 31 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/5d61fb81/phoenix-core/src/test/java/org/apache/phoenix/filter/SkipScanFilterTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/filter/SkipScanFilterTest.java b/phoenix-core/src/test/java/org/apache/phoenix/filter/SkipScanFilterTest.java
index 8ed0add..d691535 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/filter/SkipScanFilterTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/filter/SkipScanFilterTest.java
@@ -106,37 +106,37 @@ public class SkipScanFilterTest extends TestCase {
     public static Collection<Object> data() {
         List<Object> testCases = Lists.newArrayList();
         // Variable length tests
-//        testCases.addAll(
-//                foreach(new KeyRange[][]{{
-//                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true),
-//                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("e"), true, Bytes.toBytes("e"), true),
-//                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("f"), true, Bytes.toBytes("f"), true),
-//                },
-//                {
-//                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
-//                },
-//                {
-//                    KeyRange.EVERYTHING_RANGE,
-//                },
-//                {
-//                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
-//                }},
-//                new int[4],
-//                new Include(ByteUtil.concat(Bytes.toBytes("a"),QueryConstants.SEPARATOR_BYTE_ARRAY, 
-//                                            Bytes.toBytes("b"), QueryConstants.SEPARATOR_BYTE_ARRAY,
-//                                            QueryConstants.SEPARATOR_BYTE_ARRAY,
-//                                            Bytes.toBytes("1") ) ),
-//                new SeekNext(ByteUtil.concat(Bytes.toBytes("e.f"),QueryConstants.SEPARATOR_BYTE_ARRAY, 
-//                                             Bytes.toBytes("b"), QueryConstants.SEPARATOR_BYTE_ARRAY,
-//                                             QueryConstants.SEPARATOR_BYTE_ARRAY,
-//                                             Bytes.toBytes("1") ), 
-//                            ByteUtil.concat(Bytes.toBytes("f"),QueryConstants.SEPARATOR_BYTE_ARRAY, 
-//                                            Bytes.toBytes("b") )),
-//                new Include(ByteUtil.concat(Bytes.toBytes("f"),QueryConstants.SEPARATOR_BYTE_ARRAY, 
-//                                            Bytes.toBytes("b"), QueryConstants.SEPARATOR_BYTE_ARRAY,
-//                                            QueryConstants.SEPARATOR_BYTE_ARRAY,
-//                                            Bytes.toBytes("1") ) ) )
-//        );
+        testCases.addAll(
+                foreach(new KeyRange[][]{{
+                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true),
+                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("e"), true, Bytes.toBytes("e"), true),
+                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("f"), true, Bytes.toBytes("f"), true),
+                },
+                {
+                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
+                },
+                {
+                    KeyRange.EVERYTHING_RANGE,
+                },
+                {
+                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
+                }},
+                new int[4],
+                new Include(ByteUtil.concat(Bytes.toBytes("a"),QueryConstants.SEPARATOR_BYTE_ARRAY, 
+                                            Bytes.toBytes("b"), QueryConstants.SEPARATOR_BYTE_ARRAY,
+                                            QueryConstants.SEPARATOR_BYTE_ARRAY,
+                                            Bytes.toBytes("1") ) ),
+                new SeekNext(ByteUtil.concat(Bytes.toBytes("e.f"),QueryConstants.SEPARATOR_BYTE_ARRAY, 
+                                             Bytes.toBytes("b"), QueryConstants.SEPARATOR_BYTE_ARRAY,
+                                             QueryConstants.SEPARATOR_BYTE_ARRAY,
+                                             Bytes.toBytes("1") ), 
+                            ByteUtil.concat(Bytes.toBytes("f"),QueryConstants.SEPARATOR_BYTE_ARRAY, 
+                                            Bytes.toBytes("b") )),
+                new Include(ByteUtil.concat(Bytes.toBytes("f"),QueryConstants.SEPARATOR_BYTE_ARRAY, 
+                                            Bytes.toBytes("b"), QueryConstants.SEPARATOR_BYTE_ARRAY,
+                                            QueryConstants.SEPARATOR_BYTE_ARRAY,
+                                            Bytes.toBytes("1") ) ) )
+        );
         testCases.addAll(
                 foreach(new KeyRange[][]{{
                     PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("20160116121006"), true, Bytes.toBytes("20160116181006"), true),


[26/50] [abbrv] phoenix git commit: PHOENIX-2446 Immutable index - Index vs base table row count does not match when index is created during data load

Posted by ma...@apache.org.
PHOENIX-2446 Immutable index - Index vs base table row count does not match when index is created during data load


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

Branch: refs/heads/calcite
Commit: a138cfe0f3df47091a0d9fe0285a8e572d76b252
Parents: 8574d43
Author: Thomas D'Silva <td...@salesforce.com>
Authored: Fri Jan 8 15:37:31 2016 -0800
Committer: Thomas D'Silva <td...@salesforce.com>
Committed: Thu Jan 21 20:34:18 2016 -0800

----------------------------------------------------------------------
 .../phoenix/end2end/index/ImmutableIndexIT.java |  81 ++++++++---
 .../apache/phoenix/execute/PartialCommitIT.java |   2 +-
 .../cache/aggcache/SpillableGroupByCache.java   |  17 +--
 .../apache/phoenix/compile/FromCompiler.java    |   8 +-
 .../phoenix/compile/ListJarsQueryPlan.java      |   2 +-
 .../compile/PostLocalIndexDDLCompiler.java      | 104 +++++++++++++
 .../apache/phoenix/compile/TraceQueryPlan.java  |   2 +-
 .../apache/phoenix/compile/UnionCompiler.java   |   2 +-
 .../phoenix/coprocessor/BaseRegionScanner.java  |  11 +-
 .../GroupedAggregateRegionObserver.java         |  38 +----
 .../coprocessor/MetaDataEndpointImpl.java       |   7 +-
 .../coprocessor/MetaDataRegionObserver.java     |   8 +-
 .../phoenix/coprocessor/ScanRegionObserver.java |  17 +--
 .../UngroupedAggregateRegionObserver.java       |  24 +--
 .../coprocessor/generated/PTableProtos.java     | 145 +++++++++++++++----
 .../apache/phoenix/execute/MutationState.java   |  57 ++++----
 .../org/apache/phoenix/query/QueryServices.java |   2 +
 .../phoenix/query/QueryServicesOptions.java     |   9 +-
 .../apache/phoenix/schema/DelegateColumn.java   |   5 +
 .../apache/phoenix/schema/MetaDataClient.java   | 133 +++++++----------
 .../java/org/apache/phoenix/schema/PColumn.java |   2 +
 .../org/apache/phoenix/schema/PColumnImpl.java  |  21 ++-
 .../apache/phoenix/schema/PMetaDataImpl.java    |   2 +-
 .../org/apache/phoenix/schema/SaltingUtil.java  |   2 +-
 .../phoenix/execute/CorrelatePlanTest.java      |   2 +-
 .../phoenix/execute/UnnestArrayPlanTest.java    |   4 +-
 .../expression/ColumnExpressionTest.java        |   8 +-
 .../iterate/AggregateResultScannerTest.java     |   4 +
 .../phoenix/query/QueryServicesTestImpl.java    |   4 +-
 phoenix-protocol/src/main/MetaDataService.proto |   2 +-
 phoenix-protocol/src/main/PTable.proto          |   1 +
 31 files changed, 449 insertions(+), 277 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/a138cfe0/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
index 7171382..c18e4ab 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
@@ -20,6 +20,7 @@ package org.apache.phoenix.end2end.index;
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.sql.Connection;
 import java.sql.DriverManager;
@@ -34,6 +35,7 @@ import java.util.Properties;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
+import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -48,6 +50,7 @@ import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
 import org.apache.phoenix.end2end.Shadower;
+import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.query.BaseTest;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.util.PropertiesUtil;
@@ -55,6 +58,7 @@ import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.TestUtil;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
@@ -68,6 +72,7 @@ import com.google.common.collect.Maps;
 public class ImmutableIndexIT extends BaseHBaseManagedTimeIT {
 
     private final boolean localIndex;
+    private final boolean transactional;
     private final String tableDDLOptions;
     private final String tableName;
     private final String indexName;
@@ -80,6 +85,7 @@ public class ImmutableIndexIT extends BaseHBaseManagedTimeIT {
 
     public ImmutableIndexIT(boolean localIndex, boolean transactional) {
         this.localIndex = localIndex;
+        this.transactional = transactional;
         StringBuilder optionBuilder = new StringBuilder("IMMUTABLE_ROWS=true");
         if (transactional) {
             optionBuilder.append(", TRANSACTIONAL=true");
@@ -98,16 +104,55 @@ public class ImmutableIndexIT extends BaseHBaseManagedTimeIT {
         serverProps.put("hbase.coprocessor.region.classes", CreateIndexRegionObserver.class.getName());
         Map<String, String> clientProps = Maps.newHashMapWithExpectedSize(2);
         clientProps.put(QueryServices.TRANSACTIONS_ENABLED, "true");
+        clientProps.put(QueryServices.INDEX_POPULATION_SLEEP_TIME, "15000");
         setUpTestDriver(new ReadOnlyProps(serverProps.entrySet().iterator()), new ReadOnlyProps(clientProps.entrySet().iterator()));
     }
 
     @Parameters(name="localIndex = {0} , transactional = {1}")
     public static Collection<Boolean[]> data() {
-        return Arrays.asList(new Boolean[][] {     
-            { false, true }, { true, true }
-        });
+		return Arrays.asList(new Boolean[][] { 
+				{ false, false }, { false, true },
+				{ true, false }, { true, true } });
     }
 
+    @Test
+    @Ignore
+    public void testDropIfImmutableKeyValueColumn() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.setAutoCommit(false);
+            String ddl =
+                    "CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            populateTestTable(fullTableName);
+            ddl =
+                    "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON "
+                            + fullTableName + " (long_col1)";
+            stmt.execute(ddl);
+
+            ResultSet rs;
+
+            rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullTableName);
+            assertTrue(rs.next());
+            assertEquals(3, rs.getInt(1));
+            rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullIndexName);
+            assertTrue(rs.next());
+            assertEquals(3, rs.getInt(1));
+
+            conn.setAutoCommit(true);
+            String dml = "DELETE from " + fullTableName + " WHERE long_col2 = 4";
+            try {
+                conn.createStatement().execute(dml);
+                fail();
+            } catch (SQLException e) {
+                assertEquals(SQLExceptionCode.INVALID_FILTER_ON_IMMUTABLE_ROWS.getErrorCode(),
+                    e.getErrorCode());
+            }
+
+            conn.createStatement().execute("DROP TABLE " + fullTableName);
+        }
+    }
 
     @Test
     public void testCreateIndexDuringUpsertSelect() throws Exception {
@@ -119,8 +164,7 @@ public class ImmutableIndexIT extends BaseHBaseManagedTimeIT {
                 + " (long_pk, varchar_pk)"
                 + " INCLUDE (long_col1, long_col2)";
 
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
             conn.setAutoCommit(false);
             Statement stmt = conn.createStatement();
             stmt.execute(ddl);
@@ -133,7 +177,6 @@ public class ImmutableIndexIT extends BaseHBaseManagedTimeIT {
             String upsertSelect = "UPSERT INTO " + TABLE_NAME + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk) " + 
                     "SELECT varchar_pk||'_upsert_select', char_pk, int_pk, long_pk, decimal_pk, date_pk FROM "+ TABLE_NAME;    
             conn.createStatement().execute(upsertSelect);
-
             ResultSet rs;
             rs = conn.createStatement().executeQuery("SELECT /*+ NO_INDEX */ COUNT(*) FROM " + TABLE_NAME);
             assertTrue(rs.next());
@@ -142,9 +185,6 @@ public class ImmutableIndexIT extends BaseHBaseManagedTimeIT {
             assertTrue(rs.next());
             assertEquals(440,rs.getInt(1));
         }
-        finally {
-            conn.close();
-        }
     }
 
     // used to create an index while a batch of rows are being written
@@ -156,7 +196,7 @@ public class ImmutableIndexIT extends BaseHBaseManagedTimeIT {
             String tableName = c.getEnvironment().getRegion().getRegionInfo()
                     .getTable().getNameAsString();
             if (tableName.equalsIgnoreCase(TABLE_NAME)
-                    // create the index after the second batch of 1000 rows
+                    // create the index after the second batch  
                     && Bytes.startsWith(put.getRow(), Bytes.toBytes("varchar200_upsert_select"))) {
                 try {
                     Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -171,13 +211,14 @@ public class ImmutableIndexIT extends BaseHBaseManagedTimeIT {
     }
 
     private static class UpsertRunnable implements Runnable {
-        private static final int NUM_ROWS_IN_BATCH = 10000;
+        private static final int NUM_ROWS_IN_BATCH = 1000;
         private final String fullTableName;
 
         public UpsertRunnable(String fullTableName) {
             this.fullTableName = fullTableName;
         }
 
+        @Override
         public void run() {
             Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
             try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
@@ -190,12 +231,9 @@ public class ImmutableIndexIT extends BaseHBaseManagedTimeIT {
                         fistRowInBatch = false;
                     }
                     conn.commit();
-                    Thread.sleep(500);
                 }
             } catch (SQLException e) {
                 throw new RuntimeException(e);
-            } catch (InterruptedException e) {
-                Thread.currentThread().interrupt();
             }
         }
     }
@@ -213,10 +251,17 @@ public class ImmutableIndexIT extends BaseHBaseManagedTimeIT {
             Statement stmt = conn.createStatement();
             stmt.execute(ddl);
 
-            ExecutorService threadPool = Executors.newFixedThreadPool(numThreads);
+            ExecutorService executorService = Executors.newFixedThreadPool(numThreads, new ThreadFactory() {
+                @Override
+                public Thread newThread(Runnable r) {
+                    Thread t = Executors.defaultThreadFactory().newThread(r);
+                    t.setDaemon(true);
+                    return t;
+                }
+            });
             List<Future<?>> futureList = Lists.newArrayListWithExpectedSize(numThreads);
             for (int i =0; i<numThreads; ++i) {
-                futureList.add(threadPool.submit(new UpsertRunnable(fullTableName)));
+                futureList.add(executorService.submit(new UpsertRunnable(fullTableName)));
             }
             // upsert some rows before creating the index 
             Thread.sleep(500);
@@ -235,8 +280,8 @@ public class ImmutableIndexIT extends BaseHBaseManagedTimeIT {
             for (Future<?> future : futureList) {
                 future.cancel(true);
             }
-            threadPool.shutdownNow();
-            threadPool.awaitTermination(30, TimeUnit.SECONDS);
+            executorService.shutdownNow();
+            executorService.awaitTermination(30, TimeUnit.SECONDS);
             Thread.sleep(100);
 
             ResultSet rs;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a138cfe0/phoenix-core/src/it/java/org/apache/phoenix/execute/PartialCommitIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/execute/PartialCommitIT.java b/phoenix-core/src/it/java/org/apache/phoenix/execute/PartialCommitIT.java
index 8d7ebcb..0fb1869 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/execute/PartialCommitIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/execute/PartialCommitIT.java
@@ -265,7 +265,7 @@ public class PartialCommitIT extends BaseOwnClusterIT {
         Connection con = driver.connect(url, new Properties());
         PhoenixConnection phxCon = new PhoenixConnection(con.unwrap(PhoenixConnection.class));
         final Map<TableRef,Map<ImmutableBytesPtr,MutationState.RowMutationState>> mutations = Maps.newTreeMap(new TableRefComparator());
-        // passing a null mutation staate forces the connection.newMutationState() to be used to create the MutationState
+        // passing a null mutation state forces the connection.newMutationState() to be used to create the MutationState
         return new PhoenixConnection(phxCon, null) {
             @Override
             protected MutationState newMutationState(int maxSize) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a138cfe0/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillableGroupByCache.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillableGroupByCache.java b/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillableGroupByCache.java
index 69fc6f6..8edeb3a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillableGroupByCache.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillableGroupByCache.java
@@ -340,12 +340,7 @@ public class SpillableGroupByCache implements GroupByCache {
         final Iterator<Entry<ImmutableBytesWritable, Aggregator[]>> cacheIter = new EntryIterator();
 
         // scanner using the spillable implementation
-        return new BaseRegionScanner() {
-            @Override
-            public HRegionInfo getRegionInfo() {
-                return s.getRegionInfo();
-            }
-
+        return new BaseRegionScanner(s) {
             @Override
             public void close() throws IOException {
                 try {
@@ -374,16 +369,6 @@ public class SpillableGroupByCache implements GroupByCache {
                         SINGLE_COLUMN, AGG_TIMESTAMP, value, 0, value.length));
                 return cacheIter.hasNext();
             }
-
-            @Override
-            public long getMaxResultSize() {
-              return s.getMaxResultSize();
-            }
-
-            @Override
-            public int getBatch() {
-                return s.getBatch();
-            }
         };
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a138cfe0/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
index 9b2c460..dd93c81 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
@@ -221,7 +221,7 @@ public class FromCompiler {
             Expression sourceExpression = projector.getColumnProjector(column.getPosition()).getExpression();
             PColumnImpl projectedColumn = new PColumnImpl(column.getName(), column.getFamilyName(),
                     sourceExpression.getDataType(), sourceExpression.getMaxLength(), sourceExpression.getScale(), sourceExpression.isNullable(),
-                    column.getPosition(), sourceExpression.getSortOrder(), column.getArraySize(), column.getViewConstant(), column.isViewReferenced(), column.getExpressionStr(), column.isRowTimestamp());
+                    column.getPosition(), sourceExpression.getSortOrder(), column.getArraySize(), column.getViewConstant(), column.isViewReferenced(), column.getExpressionStr(), column.isRowTimestamp(), column.isDynamic());
             projectedColumns.add(projectedColumn);
         }
         PTable t = PTableImpl.makePTable(table, projectedColumns);
@@ -406,7 +406,7 @@ public class FromCompiler {
             String fullTableName = SchemaUtil.getTableName(schemaName, tableName);
             PName tenantId = connection.getTenantId();
             PTable theTable = null;
-            if (updateCacheImmediately || connection.getAutoCommit()) {
+            if (updateCacheImmediately) {
                 MetaDataMutationResult result = client.updateCache(schemaName, tableName);
                 timeStamp = TransactionUtil.getResolvedTimestamp(connection, result);
                 theTable = result.getTable();
@@ -547,7 +547,7 @@ public class FromCompiler {
                         familyName = PNameFactory.newName(family);
                     }
                     allcolumns.add(new PColumnImpl(name, familyName, dynColumn.getDataType(), dynColumn.getMaxLength(),
-                            dynColumn.getScale(), dynColumn.isNull(), position, dynColumn.getSortOrder(), dynColumn.getArraySize(), null, false, dynColumn.getExpression(), false));
+                            dynColumn.getScale(), dynColumn.isNull(), position, dynColumn.getSortOrder(), dynColumn.getArraySize(), null, false, dynColumn.getExpression(), false, true));
                     position++;
                 }
                 theTable = PTableImpl.makePTable(theTable, allcolumns);
@@ -645,7 +645,7 @@ public class FromCompiler {
                 }
                 PColumnImpl column = new PColumnImpl(PNameFactory.newName(alias),
                         PNameFactory.newName(QueryConstants.DEFAULT_COLUMN_FAMILY),
-                        null, 0, 0, true, position++, SortOrder.ASC, null, null, false, null, false);
+                        null, 0, 0, true, position++, SortOrder.ASC, null, null, false, null, false, false);
                 columns.add(column);
             }
             PTable t = PTableImpl.makePTable(null, PName.EMPTY_NAME, PName.EMPTY_NAME,

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a138cfe0/phoenix-core/src/main/java/org/apache/phoenix/compile/ListJarsQueryPlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/ListJarsQueryPlan.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/ListJarsQueryPlan.java
index dac691f..f2b4856 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/ListJarsQueryPlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/ListJarsQueryPlan.java
@@ -80,7 +80,7 @@ public class ListJarsQueryPlan implements QueryPlan {
         PColumn column =
                 new PColumnImpl(PNameFactory.newName("jar_location"), null,
                         PVarchar.INSTANCE, null, null, false, 0, SortOrder.getDefault(), 0, null,
-                        false, null, false);
+                        false, null, false, false);
         List<PColumn> columns = new ArrayList<PColumn>();
         columns.add(column);
         Expression expression =

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a138cfe0/phoenix-core/src/main/java/org/apache/phoenix/compile/PostLocalIndexDDLCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/PostLocalIndexDDLCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/PostLocalIndexDDLCompiler.java
new file mode 100644
index 0000000..f92738c
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/PostLocalIndexDDLCompiler.java
@@ -0,0 +1,104 @@
+/*
+ * 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.phoenix.compile;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.coprocessor.BaseScannerRegionObserver;
+import org.apache.phoenix.execute.MutationState;
+import org.apache.phoenix.hbase.index.covered.update.ColumnReference;
+import org.apache.phoenix.index.IndexMaintainer;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixStatement;
+import org.apache.phoenix.jdbc.PhoenixStatement.Operation;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.schema.types.PLong;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.ScanUtil;
+
+import com.google.common.collect.Lists;
+
+/**
+ * For local indexes, we optimize the initial index population by *not* sending
+ * Puts over the wire for the index rows, as we don't need to do that. Instead,
+ * we tap into our region observer to generate the index rows based on the data
+ * rows as we scan
+ */
+public class PostLocalIndexDDLCompiler {
+	private final PhoenixConnection connection;
+    private final String tableName;
+    
+    public PostLocalIndexDDLCompiler(PhoenixConnection connection, String tableName) {
+        this.connection = connection;
+        this.tableName = tableName;
+    }
+
+	public MutationPlan compile(final PTable index) throws SQLException {
+		try (final PhoenixStatement statement = new PhoenixStatement(connection)) {
+            String query = "SELECT count(*) FROM " + tableName;
+            final QueryPlan plan = statement.compileQuery(query);
+            TableRef tableRef = plan.getTableRef();
+            Scan scan = plan.getContext().getScan();
+            ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+            final PTable dataTable = tableRef.getTable();
+            List<PTable> indexes = Lists.newArrayListWithExpectedSize(1);
+            // Only build newly created index.
+            indexes.add(index);
+            IndexMaintainer.serialize(dataTable, ptr, indexes, plan.getContext().getConnection());
+            // Set attribute on scan that UngroupedAggregateRegionObserver will switch on.
+            // We'll detect that this attribute was set the server-side and write the index
+            // rows per region as a result. The value of the attribute will be our persisted
+            // index maintainers.
+            // Define the LOCAL_INDEX_BUILD as a new static in BaseScannerRegionObserver
+            scan.setAttribute(BaseScannerRegionObserver.LOCAL_INDEX_BUILD, ByteUtil.copyKeyBytesIfNecessary(ptr));
+            // By default, we'd use a FirstKeyOnly filter as nothing else needs to be projected for count(*).
+            // However, in this case, we need to project all of the data columns that contribute to the index.
+            IndexMaintainer indexMaintainer = index.getIndexMaintainer(dataTable, connection);
+            for (ColumnReference columnRef : indexMaintainer.getAllColumns()) {
+                scan.addColumn(columnRef.getFamily(), columnRef.getQualifier());
+            }
+
+            // Go through MutationPlan abstraction so that we can create local indexes
+            // with a connectionless connection (which makes testing easier).
+            return new BaseMutationPlan(plan.getContext(), Operation.UPSERT) {
+
+                @Override
+                public MutationState execute() throws SQLException {
+                    connection.getMutationState().commitDDLFence(dataTable);
+                    Cell kv = plan.iterator().next().getValue(0);
+                    ImmutableBytesWritable tmpPtr = new ImmutableBytesWritable(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength());
+                    // A single Cell will be returned with the count(*) - we decode that here
+                    long rowCount = PLong.INSTANCE.getCodec().decodeLong(tmpPtr, SortOrder.getDefault());
+                    // The contract is to return a MutationState that contains the number of rows modified. In this
+                    // case, it's the number of rows in the data table which corresponds to the number of index
+                    // rows that were added.
+                    return new MutationState(0, connection, rowCount);
+                }
+
+            };
+        }
+	}
+	
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a138cfe0/phoenix-core/src/main/java/org/apache/phoenix/compile/TraceQueryPlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/TraceQueryPlan.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/TraceQueryPlan.java
index a9754b3..1e8210a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/TraceQueryPlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/TraceQueryPlan.java
@@ -77,7 +77,7 @@ public class TraceQueryPlan implements QueryPlan {
         PColumn column =
                 new PColumnImpl(PNameFactory.newName(MetricInfo.TRACE.columnName), null,
                         PLong.INSTANCE, null, null, false, 0, SortOrder.getDefault(), 0, null,
-                        false, null, false);
+                        false, null, false, false);
         List<PColumn> columns = new ArrayList<PColumn>();
         columns.add(column);
         Expression expression =

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a138cfe0/phoenix-core/src/main/java/org/apache/phoenix/compile/UnionCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/UnionCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/UnionCompiler.java
index 3bc1e37..f8b2778 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/UnionCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/UnionCompiler.java
@@ -75,7 +75,7 @@ public class UnionCompiler {
             String name = selectNodes == null ? colProj.getName() : selectNodes.get(i).getAlias();
             PColumnImpl projectedColumn = new PColumnImpl(PNameFactory.newName(name), UNION_FAMILY_NAME,
                     sourceExpression.getDataType(), sourceExpression.getMaxLength(), sourceExpression.getScale(), sourceExpression.isNullable(),
-                    i, sourceExpression.getSortOrder(), 500, null, false, sourceExpression.toString(), false);
+                    i, sourceExpression.getSortOrder(), 500, null, false, sourceExpression.toString(), false, false);
             projectedColumns.add(projectedColumn);
         }
         Long scn = statement.getConnection().getSCN();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a138cfe0/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseRegionScanner.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseRegionScanner.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseRegionScanner.java
index 3f73048..b5e9c9f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseRegionScanner.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseRegionScanner.java
@@ -25,8 +25,12 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.hadoop.hbase.regionserver.ScannerContext;
 
-public abstract class BaseRegionScanner implements RegionScanner {
+public abstract class BaseRegionScanner extends DelegateRegionScanner {
 
+	public BaseRegionScanner(RegionScanner delegate) {
+		super(delegate);
+	}
+	
     @Override
     public boolean isFilterDone() {
         return false;
@@ -46,11 +50,6 @@ public abstract class BaseRegionScanner implements RegionScanner {
     }
 
     @Override
-    public long getMvccReadPoint() {
-        return Long.MAX_VALUE;
-    }
-
-    @Override
     public boolean nextRaw(List<Cell> result) throws IOException {
         return next(result);
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a138cfe0/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java
index d613688..3237882 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java
@@ -315,15 +315,10 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
                 aggResults.add(keyValue);
             }
             // scanner using the non spillable, memory-only implementation
-            return new BaseRegionScanner() {
+            return new BaseRegionScanner(s) {
                 private int index = 0;
 
                 @Override
-                public HRegionInfo getRegionInfo() {
-                    return s.getRegionInfo();
-                }
-
-                @Override
                 public void close() throws IOException {
                     try {
                         s.close();
@@ -341,16 +336,6 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
                     index++;
                     return index < aggResults.size();
                 }
-
-                @Override
-                public long getMaxResultSize() {
-                	return s.getMaxResultSize();
-                }
-
-                @Override
-                public int getBatch() {
-                    return s.getBatch();
-                }
             };
         }
 
@@ -471,21 +456,11 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
             logger.debug(LogUtil.addCustomAnnotations("Grouped aggregation over ordered rows with scan " + scan + ", group by "
                     + expressions + ", aggregators " + aggregators, ScanUtil.getCustomAnnotations(scan)));
         }
-        return new BaseRegionScanner() {
+        return new BaseRegionScanner(scanner) {
             private long rowCount = 0;
             private ImmutableBytesWritable currentKey = null;
 
             @Override
-            public HRegionInfo getRegionInfo() {
-                return scanner.getRegionInfo();
-            }
-
-            @Override
-            public void close() throws IOException {
-                scanner.close();
-            }
-
-            @Override
             public boolean next(List<Cell> results) throws IOException {
                 boolean hasMore;
                 boolean atLimit;
@@ -567,15 +542,6 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
                 currentKey = null;
                 return false;
             }
-
-            @Override
-            public long getMaxResultSize() {
-                return scanner.getMaxResultSize();
-            }
-            @Override
-            public int getBatch() {
-                return scanner.getBatch();
-            }
         };
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a138cfe0/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
----------------------------------------------------------------------
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 9887e7b..78f9700 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
@@ -91,6 +91,7 @@ import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
@@ -109,9 +110,12 @@ import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
 import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.regionserver.MultiVersionConsistencyControl;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.Region.RowLock;
+import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.regionserver.RegionServerServices;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.cache.GlobalCache;
@@ -188,6 +192,7 @@ import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.ServerUtil;
 import org.apache.phoenix.util.UpgradeUtil;
+import org.hamcrest.core.IsInstanceOf;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -620,7 +625,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                         isRowTimestampKV.getValueArray(), isRowTimestampKV.getValueOffset(),
                         isRowTimestampKV.getValueLength()));
 
-        PColumn column = new PColumnImpl(colName, famName, dataType, maxLength, scale, isNullable, position-1, sortOrder, arraySize, viewConstant, isViewReferenced, expressionStr, isRowTimestamp);
+        PColumn column = new PColumnImpl(colName, famName, dataType, maxLength, scale, isNullable, position-1, sortOrder, arraySize, viewConstant, isViewReferenced, expressionStr, isRowTimestamp, false);
         columns.add(column);
     }
     

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a138cfe0/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
index 7950ac8..a2f7282 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
@@ -22,6 +22,7 @@ import java.sql.DriverManager;
 import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Properties;
 import java.util.TimerTask;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
@@ -252,7 +253,10 @@ public class MetaDataRegionObserver extends BaseRegionObserver {
                     }
 
                     if (conn == null) {
-                        conn = DriverManager.getConnection(getJdbcUrl(env)).unwrap(PhoenixConnection.class);
+                    	final Properties props = new Properties();
+                    	// don't run a second index populations upsert select 
+                        props.setProperty(QueryServices.INDEX_POPULATION_SLEEP_TIME, "0"); 
+                        conn = DriverManager.getConnection(getJdbcUrl(env), props).unwrap(PhoenixConnection.class);
                     }
 
                     String dataTableFullName = SchemaUtil.getTableName(schemaName, dataTable);
@@ -270,7 +274,7 @@ public class MetaDataRegionObserver extends BaseRegionObserver {
                         QueryServices.INDEX_FAILURE_HANDLING_REBUILD_OVERLAP_TIME_ATTRIB,
                         QueryServicesOptions.DEFAULT_INDEX_FAILURE_HANDLING_REBUILD_OVERLAP_TIME);
                     long timeStamp = Math.max(0, disabledTimeStampVal - overlapTime);
-
+                    
                     LOG.info("Starting to build index=" + indexPTable.getName() + " from timestamp=" + timeStamp);
                     client.buildPartialIndexFromTimeStamp(indexPTable, new TableRef(dataPTable, Long.MAX_VALUE, timeStamp));
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a138cfe0/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ScanRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ScanRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ScanRegionObserver.java
index 65a43de..d2bd3b3 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ScanRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ScanRegionObserver.java
@@ -253,7 +253,7 @@ public class ScanRegionObserver extends BaseScannerRegionObserver {
         } finally {
             region.closeRegionOperation();
         }
-        return new BaseRegionScanner() {
+        return new BaseRegionScanner(s) {
             private Tuple tuple = firstTuple;
 
             @Override
@@ -262,11 +262,6 @@ public class ScanRegionObserver extends BaseScannerRegionObserver {
             }
 
             @Override
-            public HRegionInfo getRegionInfo() {
-                return s.getRegionInfo();
-            }
-
-            @Override
             public boolean next(List<Cell> results) throws IOException {
                 try {
                     if (isFilterDone()) {
@@ -301,16 +296,6 @@ public class ScanRegionObserver extends BaseScannerRegionObserver {
                     }
                 }
             }
-
-            @Override
-            public long getMaxResultSize() {
-                return s.getMaxResultSize();
-            }
-
-            @Override
-            public int getBatch() {
-              return s.getBatch();
-            }
         };
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a138cfe0/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
index f332e60..05cf08e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
@@ -554,25 +554,15 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
         }
         final KeyValue aggKeyValue = keyValue;
 
-        RegionScanner scanner = new BaseRegionScanner() {
+        RegionScanner scanner = new BaseRegionScanner(innerScanner) {
             private boolean done = !hadAny;
 
             @Override
-            public HRegionInfo getRegionInfo() {
-                return innerScanner.getRegionInfo();
-            }
-
-            @Override
             public boolean isFilterDone() {
                 return done;
             }
 
             @Override
-            public void close() throws IOException {
-                innerScanner.close();
-            }
-
-            @Override
             public boolean next(List<Cell> results) throws IOException {
                 if (done) return false;
                 done = true;
@@ -584,11 +574,6 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
             public long getMaxResultSize() {
                 return scan.getMaxResultSize();
             }
-
-            @Override
-            public int getBatch() {
-                return innerScanner.getBatch();
-            }
         };
         return scanner;
     }
@@ -690,7 +675,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
         final KeyValue aggKeyValue =
                 KeyValueUtil.newKeyValue(UNGROUPED_AGG_ROW_KEY, SINGLE_COLUMN_FAMILY,
                     SINGLE_COLUMN, AGG_TIMESTAMP, rowCountBytes, 0, rowCountBytes.length);
-        RegionScanner scanner = new BaseRegionScanner() {
+        RegionScanner scanner = new BaseRegionScanner(innerScanner) {
             @Override
             public HRegionInfo getRegionInfo() {
                 return region.getRegionInfo();
@@ -716,11 +701,6 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
             public long getMaxResultSize() {
                 return scan.getMaxResultSize();
             }
-
-            @Override
-            public int getBatch() {
-                return innerScanner.getBatch();
-            }
         };
         return scanner;
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a138cfe0/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java
index be8d7e2..f74ed0b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java
@@ -259,6 +259,16 @@ public final class PTableProtos {
      * <code>optional bool isRowTimestamp = 13;</code>
      */
     boolean getIsRowTimestamp();
+
+    // optional bool isDynamic = 14;
+    /**
+     * <code>optional bool isDynamic = 14;</code>
+     */
+    boolean hasIsDynamic();
+    /**
+     * <code>optional bool isDynamic = 14;</code>
+     */
+    boolean getIsDynamic();
   }
   /**
    * Protobuf type {@code PColumn}
@@ -376,6 +386,11 @@ public final class PTableProtos {
               isRowTimestamp_ = input.readBool();
               break;
             }
+            case 112: {
+              bitField0_ |= 0x00002000;
+              isDynamic_ = input.readBool();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -678,6 +693,22 @@ public final class PTableProtos {
       return isRowTimestamp_;
     }
 
+    // optional bool isDynamic = 14;
+    public static final int ISDYNAMIC_FIELD_NUMBER = 14;
+    private boolean isDynamic_;
+    /**
+     * <code>optional bool isDynamic = 14;</code>
+     */
+    public boolean hasIsDynamic() {
+      return ((bitField0_ & 0x00002000) == 0x00002000);
+    }
+    /**
+     * <code>optional bool isDynamic = 14;</code>
+     */
+    public boolean getIsDynamic() {
+      return isDynamic_;
+    }
+
     private void initFields() {
       columnNameBytes_ = com.google.protobuf.ByteString.EMPTY;
       familyNameBytes_ = com.google.protobuf.ByteString.EMPTY;
@@ -692,6 +723,7 @@ public final class PTableProtos {
       viewReferenced_ = false;
       expression_ = "";
       isRowTimestamp_ = false;
+      isDynamic_ = false;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -764,6 +796,9 @@ public final class PTableProtos {
       if (((bitField0_ & 0x00001000) == 0x00001000)) {
         output.writeBool(13, isRowTimestamp_);
       }
+      if (((bitField0_ & 0x00002000) == 0x00002000)) {
+        output.writeBool(14, isDynamic_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -825,6 +860,10 @@ public final class PTableProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeBoolSize(13, isRowTimestamp_);
       }
+      if (((bitField0_ & 0x00002000) == 0x00002000)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(14, isDynamic_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -913,6 +952,11 @@ public final class PTableProtos {
         result = result && (getIsRowTimestamp()
             == other.getIsRowTimestamp());
       }
+      result = result && (hasIsDynamic() == other.hasIsDynamic());
+      if (hasIsDynamic()) {
+        result = result && (getIsDynamic()
+            == other.getIsDynamic());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -978,6 +1022,10 @@ public final class PTableProtos {
         hash = (37 * hash) + ISROWTIMESTAMP_FIELD_NUMBER;
         hash = (53 * hash) + hashBoolean(getIsRowTimestamp());
       }
+      if (hasIsDynamic()) {
+        hash = (37 * hash) + ISDYNAMIC_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getIsDynamic());
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -1113,6 +1161,8 @@ public final class PTableProtos {
         bitField0_ = (bitField0_ & ~0x00000800);
         isRowTimestamp_ = false;
         bitField0_ = (bitField0_ & ~0x00001000);
+        isDynamic_ = false;
+        bitField0_ = (bitField0_ & ~0x00002000);
         return this;
       }
 
@@ -1193,6 +1243,10 @@ public final class PTableProtos {
           to_bitField0_ |= 0x00001000;
         }
         result.isRowTimestamp_ = isRowTimestamp_;
+        if (((from_bitField0_ & 0x00002000) == 0x00002000)) {
+          to_bitField0_ |= 0x00002000;
+        }
+        result.isDynamic_ = isDynamic_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -1252,6 +1306,9 @@ public final class PTableProtos {
         if (other.hasIsRowTimestamp()) {
           setIsRowTimestamp(other.getIsRowTimestamp());
         }
+        if (other.hasIsDynamic()) {
+          setIsDynamic(other.getIsDynamic());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -1819,6 +1876,39 @@ public final class PTableProtos {
         return this;
       }
 
+      // optional bool isDynamic = 14;
+      private boolean isDynamic_ ;
+      /**
+       * <code>optional bool isDynamic = 14;</code>
+       */
+      public boolean hasIsDynamic() {
+        return ((bitField0_ & 0x00002000) == 0x00002000);
+      }
+      /**
+       * <code>optional bool isDynamic = 14;</code>
+       */
+      public boolean getIsDynamic() {
+        return isDynamic_;
+      }
+      /**
+       * <code>optional bool isDynamic = 14;</code>
+       */
+      public Builder setIsDynamic(boolean value) {
+        bitField0_ |= 0x00002000;
+        isDynamic_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bool isDynamic = 14;</code>
+       */
+      public Builder clearIsDynamic() {
+        bitField0_ = (bitField0_ & ~0x00002000);
+        isDynamic_ = false;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:PColumn)
     }
 
@@ -6909,38 +6999,39 @@ public final class PTableProtos {
       descriptor;
   static {
     java.lang.String[] descriptorData = {
-      "\n\014PTable.proto\032\021PGuidePosts.proto\"\223\002\n\007PC" +
+      "\n\014PTable.proto\032\021PGuidePosts.proto\"\246\002\n\007PC" +
       "olumn\022\027\n\017columnNameBytes\030\001 \002(\014\022\027\n\017family" +
       "NameBytes\030\002 \001(\014\022\020\n\010dataType\030\003 \002(\t\022\021\n\tmax" +
       "Length\030\004 \001(\005\022\r\n\005scale\030\005 \001(\005\022\020\n\010nullable\030" +
       "\006 \002(\010\022\020\n\010position\030\007 \002(\005\022\021\n\tsortOrder\030\010 \002" +
       "(\005\022\021\n\tarraySize\030\t \001(\005\022\024\n\014viewConstant\030\n " +
       "\001(\014\022\026\n\016viewReferenced\030\013 \001(\010\022\022\n\nexpressio" +
-      "n\030\014 \001(\t\022\026\n\016isRowTimestamp\030\r \001(\010\"\232\001\n\013PTab" +
-      "leStats\022\013\n\003key\030\001 \002(\014\022\016\n\006values\030\002 \003(\014\022\033\n\023" +
-      "guidePostsByteCount\030\003 \001(\003\022\025\n\rkeyBytesCou",
-      "nt\030\004 \001(\003\022\027\n\017guidePostsCount\030\005 \001(\005\022!\n\013pGu" +
-      "idePosts\030\006 \001(\0132\014.PGuidePosts\"\244\005\n\006PTable\022" +
-      "\027\n\017schemaNameBytes\030\001 \002(\014\022\026\n\016tableNameByt" +
-      "es\030\002 \002(\014\022\036\n\ttableType\030\003 \002(\0162\013.PTableType" +
-      "\022\022\n\nindexState\030\004 \001(\t\022\026\n\016sequenceNumber\030\005" +
-      " \002(\003\022\021\n\ttimeStamp\030\006 \002(\003\022\023\n\013pkNameBytes\030\007" +
-      " \001(\014\022\021\n\tbucketNum\030\010 \002(\005\022\031\n\007columns\030\t \003(\013" +
-      "2\010.PColumn\022\030\n\007indexes\030\n \003(\0132\007.PTable\022\027\n\017" +
-      "isImmutableRows\030\013 \002(\010\022 \n\nguidePosts\030\014 \003(" +
-      "\0132\014.PTableStats\022\032\n\022dataTableNameBytes\030\r ",
-      "\001(\014\022\031\n\021defaultFamilyName\030\016 \001(\014\022\022\n\ndisabl" +
-      "eWAL\030\017 \002(\010\022\023\n\013multiTenant\030\020 \002(\010\022\020\n\010viewT" +
-      "ype\030\021 \001(\014\022\025\n\rviewStatement\030\022 \001(\014\022\025\n\rphys" +
-      "icalNames\030\023 \003(\014\022\020\n\010tenantId\030\024 \001(\014\022\023\n\013vie" +
-      "wIndexId\030\025 \001(\005\022\021\n\tindexType\030\026 \001(\014\022\026\n\016sta" +
-      "tsTimeStamp\030\027 \001(\003\022\022\n\nstoreNulls\030\030 \001(\010\022\027\n" +
-      "\017baseColumnCount\030\031 \001(\005\022\036\n\026rowKeyOrderOpt" +
-      "imizable\030\032 \001(\010\022\025\n\rtransactional\030\033 \001(\010\022\034\n" +
-      "\024updateCacheFrequency\030\034 \001(\003*A\n\nPTableTyp" +
-      "e\022\n\n\006SYSTEM\020\000\022\010\n\004USER\020\001\022\010\n\004VIEW\020\002\022\t\n\005IND",
-      "EX\020\003\022\010\n\004JOIN\020\004B@\n(org.apache.phoenix.cop" +
-      "rocessor.generatedB\014PTableProtosH\001\210\001\001\240\001\001"
+      "n\030\014 \001(\t\022\026\n\016isRowTimestamp\030\r \001(\010\022\021\n\tisDyn" +
+      "amic\030\016 \001(\010\"\232\001\n\013PTableStats\022\013\n\003key\030\001 \002(\014\022" +
+      "\016\n\006values\030\002 \003(\014\022\033\n\023guidePostsByteCount\030\003",
+      " \001(\003\022\025\n\rkeyBytesCount\030\004 \001(\003\022\027\n\017guidePost" +
+      "sCount\030\005 \001(\005\022!\n\013pGuidePosts\030\006 \001(\0132\014.PGui" +
+      "dePosts\"\244\005\n\006PTable\022\027\n\017schemaNameBytes\030\001 " +
+      "\002(\014\022\026\n\016tableNameBytes\030\002 \002(\014\022\036\n\ttableType" +
+      "\030\003 \002(\0162\013.PTableType\022\022\n\nindexState\030\004 \001(\t\022" +
+      "\026\n\016sequenceNumber\030\005 \002(\003\022\021\n\ttimeStamp\030\006 \002" +
+      "(\003\022\023\n\013pkNameBytes\030\007 \001(\014\022\021\n\tbucketNum\030\010 \002" +
+      "(\005\022\031\n\007columns\030\t \003(\0132\010.PColumn\022\030\n\007indexes" +
+      "\030\n \003(\0132\007.PTable\022\027\n\017isImmutableRows\030\013 \002(\010" +
+      "\022 \n\nguidePosts\030\014 \003(\0132\014.PTableStats\022\032\n\022da",
+      "taTableNameBytes\030\r \001(\014\022\031\n\021defaultFamilyN" +
+      "ame\030\016 \001(\014\022\022\n\ndisableWAL\030\017 \002(\010\022\023\n\013multiTe" +
+      "nant\030\020 \002(\010\022\020\n\010viewType\030\021 \001(\014\022\025\n\rviewStat" +
+      "ement\030\022 \001(\014\022\025\n\rphysicalNames\030\023 \003(\014\022\020\n\010te" +
+      "nantId\030\024 \001(\014\022\023\n\013viewIndexId\030\025 \001(\005\022\021\n\tind" +
+      "exType\030\026 \001(\014\022\026\n\016statsTimeStamp\030\027 \001(\003\022\022\n\n" +
+      "storeNulls\030\030 \001(\010\022\027\n\017baseColumnCount\030\031 \001(" +
+      "\005\022\036\n\026rowKeyOrderOptimizable\030\032 \001(\010\022\025\n\rtra" +
+      "nsactional\030\033 \001(\010\022\034\n\024updateCacheFrequency" +
+      "\030\034 \001(\003*A\n\nPTableType\022\n\n\006SYSTEM\020\000\022\010\n\004USER",
+      "\020\001\022\010\n\004VIEW\020\002\022\t\n\005INDEX\020\003\022\010\n\004JOIN\020\004B@\n(org" +
+      ".apache.phoenix.coprocessor.generatedB\014P" +
+      "TableProtosH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -6952,7 +7043,7 @@ public final class PTableProtos {
           internal_static_PColumn_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_PColumn_descriptor,
-              new java.lang.String[] { "ColumnNameBytes", "FamilyNameBytes", "DataType", "MaxLength", "Scale", "Nullable", "Position", "SortOrder", "ArraySize", "ViewConstant", "ViewReferenced", "Expression", "IsRowTimestamp", });
+              new java.lang.String[] { "ColumnNameBytes", "FamilyNameBytes", "DataType", "MaxLength", "Scale", "Nullable", "Position", "SortOrder", "ArraySize", "ViewConstant", "ViewReferenced", "Expression", "IsRowTimestamp", "IsDynamic", });
           internal_static_PTableStats_descriptor =
             getDescriptor().getMessageTypes().get(1);
           internal_static_PTableStats_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a138cfe0/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
index 0a5b053..46aa819 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
@@ -54,6 +54,7 @@ import org.apache.phoenix.coprocessor.MetaDataProtocol.MetaDataMutationResult;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.hbase.index.util.KeyValueBuilder;
 import org.apache.phoenix.index.IndexMaintainer;
 import org.apache.phoenix.index.IndexMetaDataCacheClient;
 import org.apache.phoenix.index.PhoenixIndexCodec;
@@ -569,7 +570,7 @@ public class MutationState implements SQLCloseable {
                 List<Mutation> indexMutations;
                 try {
                     indexMutations =
-                            IndexUtil.generateIndexData(table, index, mutationsPertainingToIndex,
+                    		IndexUtil.generateIndexData(table, index, mutationsPertainingToIndex,
                                 connection.getKeyValueBuilder(), connection);
                     // we may also have to include delete mutations for immutable tables if we are not processing all the tables in the mutations map
                     if (!sendAll) {
@@ -719,37 +720,35 @@ public class MutationState implements SQLCloseable {
         long serverTimeStamp = tableRef.getTimeStamp();
         // If we're auto committing, we've already validated the schema when we got the ColumnResolver,
         // so no need to do it again here.
-        if (!connection.getAutoCommit()) {
-            PTable table = tableRef.getTable();
-            MetaDataMutationResult result = client.updateCache(table.getSchemaName().getString(), table.getTableName().getString());
-            PTable resolvedTable = result.getTable();
-            if (resolvedTable == null) {
-                throw new TableNotFoundException(table.getSchemaName().getString(), table.getTableName().getString());
-            }
-            // Always update tableRef table as the one we've cached may be out of date since when we executed
-            // the UPSERT VALUES call and updated in the cache before this.
-            tableRef.setTable(resolvedTable);
-            long timestamp = result.getMutationTime();
-            if (timestamp != QueryConstants.UNSET_TIMESTAMP) {
-                serverTimeStamp = timestamp;
-                if (result.wasUpdated()) {
-                    // TODO: use bitset?
-                    PColumn[] columns = new PColumn[resolvedTable.getColumns().size()];
-                    for (Map.Entry<ImmutableBytesPtr,RowMutationState> rowEntry : rowKeyToColumnMap.entrySet()) {
-                        RowMutationState valueEntry = rowEntry.getValue();
-                        if (valueEntry != null) {
-                            Map<PColumn, byte[]> colValues = valueEntry.getColumnValues();
-                            if (colValues != PRow.DELETE_MARKER) {
-                                for (PColumn column : colValues.keySet()) {
-                                    columns[column.getPosition()] = column;
-                                }
+        PTable table = tableRef.getTable();
+        MetaDataMutationResult result = client.updateCache(table.getSchemaName().getString(), table.getTableName().getString());
+        PTable resolvedTable = result.getTable();
+        if (resolvedTable == null) {
+            throw new TableNotFoundException(table.getSchemaName().getString(), table.getTableName().getString());
+        }
+        // Always update tableRef table as the one we've cached may be out of date since when we executed
+        // the UPSERT VALUES call and updated in the cache before this.
+        tableRef.setTable(resolvedTable);
+        long timestamp = result.getMutationTime();
+        if (timestamp != QueryConstants.UNSET_TIMESTAMP) {
+            serverTimeStamp = timestamp;
+            if (result.wasUpdated()) {
+                List<PColumn> columns = Lists.newArrayListWithExpectedSize(table.getColumns().size());
+                for (Map.Entry<ImmutableBytesPtr,RowMutationState> rowEntry : rowKeyToColumnMap.entrySet()) {
+                    RowMutationState valueEntry = rowEntry.getValue();
+                    if (valueEntry != null) {
+                        Map<PColumn, byte[]> colValues = valueEntry.getColumnValues();
+                        if (colValues != PRow.DELETE_MARKER) {
+                            for (PColumn column : colValues.keySet()) {
+                            	if (!column.isDynamic())
+                            		columns.add(column);
                             }
                         }
                     }
-                    for (PColumn column : columns) {
-                        if (column != null) {
-                            resolvedTable.getColumnFamily(column.getFamilyName().getString()).getColumn(column.getName().getString());
-                        }
+                }
+                for (PColumn column : columns) {
+                    if (column != null) {
+                        resolvedTable.getColumnFamily(column.getFamilyName().getString()).getColumn(column.getName().getString());
                     }
                 }
             }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a138cfe0/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 d40a15b..b0e7b6e 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
@@ -205,6 +205,8 @@ public interface QueryServices extends SQLCloseable {
     public static final String HCONNECTION_POOL_MAX_SIZE = "hbase.hconnection.threads.max";
     public static final String HTABLE_MAX_THREADS = "hbase.htable.threads.max";
 
+    // time to wait before running second index population upsert select (so that any pending batches of rows on region server are also written to index)
+    public static final String INDEX_POPULATION_SLEEP_TIME = "phoenix.index.population.wait.time";
     /**
      * Get executor service used for parallel scans
      */

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a138cfe0/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 9257413..c9bc19b 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
@@ -37,6 +37,7 @@ import static org.apache.phoenix.query.QueryServices.GROUPBY_SPILLABLE_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.GROUPBY_SPILL_FILES_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.IMMUTABLE_ROWS_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.INDEX_MUTATE_BATCH_SIZE_THRESHOLD_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.INDEX_POPULATION_SLEEP_TIME;
 import static org.apache.phoenix.query.QueryServices.KEEP_ALIVE_MS_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.MASTER_INFO_PORT_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.MAX_CLIENT_METADATA_CACHE_SIZE_ATTRIB;
@@ -223,6 +224,8 @@ public class QueryServicesOptions {
     
     public static final boolean DEFAULT_RETURN_SEQUENCE_VALUES = false;
     public static final String DEFAULT_EXTRA_JDBC_ARGUMENTS = "";
+    
+    public static final long DEFAULT_INDEX_POPULATION_SLEEP_TIME = 5000;
 
     // QueryServer defaults -- ensure ThinClientUtil is also updated since phoenix-server-client
     // doesn't depend on phoenix-core.
@@ -430,7 +433,6 @@ public class QueryServicesOptions {
         return set(GROUPBY_SPILL_FILES_ATTRIB, num);
     }
 
-
     private QueryServicesOptions set(String name, boolean value) {
         config.set(name, Boolean.toString(value));
         return this;
@@ -641,4 +643,9 @@ public class QueryServicesOptions {
         return this;
     }
     
+    public QueryServicesOptions setDefaultIndexPopulationWaitTime(long waitTime) {
+        config.setLong(INDEX_POPULATION_SLEEP_TIME, waitTime);
+        return this;
+    }
+    
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a138cfe0/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateColumn.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateColumn.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateColumn.java
index ddb0a1a..798706e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateColumn.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateColumn.java
@@ -84,4 +84,9 @@ public class DelegateColumn extends DelegateDatum implements PColumn {
     public String toString() {
         return getDelegate().toString();
     }
+
+	@Override
+	public boolean isDynamic() {
+		return getDelegate().isDynamic();
+	}
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a138cfe0/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 d559842..e8d995c 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
@@ -114,9 +114,6 @@ import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
 
-import co.cask.tephra.TxConstants;
-
-import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionLocation;
@@ -125,17 +122,15 @@ import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
-import org.apache.phoenix.compile.BaseMutationPlan;
 import org.apache.phoenix.compile.ColumnResolver;
 import org.apache.phoenix.compile.FromCompiler;
 import org.apache.phoenix.compile.IndexExpressionCompiler;
 import org.apache.phoenix.compile.MutationPlan;
 import org.apache.phoenix.compile.PostDDLCompiler;
 import org.apache.phoenix.compile.PostIndexDDLCompiler;
-import org.apache.phoenix.compile.QueryPlan;
+import org.apache.phoenix.compile.PostLocalIndexDDLCompiler;
 import org.apache.phoenix.compile.StatementContext;
 import org.apache.phoenix.compile.StatementNormalizer;
 import org.apache.phoenix.coprocessor.BaseScannerRegionObserver;
@@ -153,7 +148,6 @@ import org.apache.phoenix.index.IndexMaintainer;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.jdbc.PhoenixStatement;
-import org.apache.phoenix.jdbc.PhoenixStatement.Operation;
 import org.apache.phoenix.parse.AddColumnStatement;
 import org.apache.phoenix.parse.AlterIndexStatement;
 import org.apache.phoenix.parse.ColumnDef;
@@ -210,6 +204,8 @@ import org.apache.phoenix.util.UpgradeUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import co.cask.tephra.TxConstants;
+
 import com.google.common.base.Objects;
 import com.google.common.collect.Iterators;
 import com.google.common.collect.ListMultimap;
@@ -821,7 +817,7 @@ public class MetaDataClient {
             }
 
             PColumn column = new PColumnImpl(PNameFactory.newName(columnName), familyName, def.getDataType(),
-                    def.getMaxLength(), def.getScale(), isNull, position, sortOrder, def.getArraySize(), null, false, def.getExpression(), isRowTimestamp);
+                    def.getMaxLength(), def.getScale(), isNull, position, sortOrder, def.getArraySize(), null, false, def.getExpression(), isRowTimestamp, false);
             return column;
         } catch (IllegalArgumentException e) { // Based on precondition check in constructor
             throw new SQLException(e);
@@ -1021,89 +1017,68 @@ public class MetaDataClient {
         try {
             connection.setAutoCommit(true);
             MutationPlan mutationPlan;
-
-            // For local indexes, we optimize the initial index population by *not* sending Puts over
-            // the wire for the index rows, as we don't need to do that. Instead, we tap into our
-            // region observer to generate the index rows based on the data rows as we scan
             if (index.getIndexType() == IndexType.LOCAL) {
-                try (final PhoenixStatement statement = new PhoenixStatement(connection)) {
-                    String tableName = getFullTableName(dataTableRef);
-                    String query = "SELECT count(*) FROM " + tableName;
-                    final QueryPlan plan = statement.compileQuery(query);
-                    TableRef tableRef = plan.getTableRef();
-                    // Set attribute on scan that UngroupedAggregateRegionObserver will switch on.
-                    // We'll detect that this attribute was set the server-side and write the index
-                    // rows per region as a result. The value of the attribute will be our persisted
-                    // index maintainers.
-                    // Define the LOCAL_INDEX_BUILD as a new static in BaseScannerRegionObserver
-                    Scan scan = plan.getContext().getScan();
-                    try {
-                        if(ScanUtil.isDefaultTimeRange(scan.getTimeRange())) {
-                            Long scn = connection.getSCN();
-                            if (scn == null) {
-                                scn = plan.getContext().getCurrentTime();
-                            }
-                            scan.setTimeRange(dataTableRef.getLowerBoundTimeStamp(),scn);
-                        }
-                    } catch (IOException e) {
-                        throw new SQLException(e);
-                    }
-                    ImmutableBytesWritable ptr = new ImmutableBytesWritable();
-                    final PTable dataTable = tableRef.getTable();
-                    for(PTable idx: dataTable.getIndexes()) {
-                        if(idx.getName().equals(index.getName())) {
-                            index = idx;
-                            break;
-                        }
-                    }
-                    List<PTable> indexes = Lists.newArrayListWithExpectedSize(1);
-                    // Only build newly created index.
-                    indexes.add(index);
-                    IndexMaintainer.serialize(dataTable, ptr, indexes, plan.getContext().getConnection());
-                    scan.setAttribute(BaseScannerRegionObserver.LOCAL_INDEX_BUILD, ByteUtil.copyKeyBytesIfNecessary(ptr));
-                    // By default, we'd use a FirstKeyOnly filter as nothing else needs to be projected for count(*).
-                    // However, in this case, we need to project all of the data columns that contribute to the index.
-                    IndexMaintainer indexMaintainer = index.getIndexMaintainer(dataTable, connection);
-                    for (ColumnReference columnRef : indexMaintainer.getAllColumns()) {
-                        scan.addColumn(columnRef.getFamily(), columnRef.getQualifier());
-                    }
-
-                    // Go through MutationPlan abstraction so that we can create local indexes
-                    // with a connectionless connection (which makes testing easier).
-                    mutationPlan = new BaseMutationPlan(plan.getContext(), Operation.UPSERT) {
-
-                        @Override
-                        public MutationState execute() throws SQLException {
-                            connection.getMutationState().commitDDLFence(dataTable);
-                            Cell kv = plan.iterator().next().getValue(0);
-                            ImmutableBytesWritable tmpPtr = new ImmutableBytesWritable(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength());
-                            // A single Cell will be returned with the count(*) - we decode that here
-                            long rowCount = PLong.INSTANCE.getCodec().decodeLong(tmpPtr, SortOrder.getDefault());
-                            // The contract is to return a MutationState that contains the number of rows modified. In this
-                            // case, it's the number of rows in the data table which corresponds to the number of index
-                            // rows that were added.
-                            return new MutationState(0, connection, rowCount);
-                        }
-
-                    };
-                }
+                PostLocalIndexDDLCompiler compiler =
+                        new PostLocalIndexDDLCompiler(connection, getFullTableName(dataTableRef));
+                mutationPlan = compiler.compile(index);
             } else {
                 PostIndexDDLCompiler compiler = new PostIndexDDLCompiler(connection, dataTableRef);
                 mutationPlan = compiler.compile(index);
-                try {
-                    Long scn = connection.getSCN();
+            }
+            Scan scan = mutationPlan.getContext().getScan();
+            Long scn = connection.getSCN();
+            try {
+                if (ScanUtil.isDefaultTimeRange(scan.getTimeRange())) {
                     if (scn == null) {
                         scn = mutationPlan.getContext().getCurrentTime();
                     }
-                    mutationPlan.getContext().getScan().setTimeRange(dataTableRef.getLowerBoundTimeStamp(), scn);
+                    scan.setTimeRange(dataTableRef.getLowerBoundTimeStamp(), scn);
+                }
+            } catch (IOException e) {
+                throw new SQLException(e);
+            }
+            
+            // execute index population upsert select
+            long startTime = System.currentTimeMillis();
+            MutationState state = connection.getQueryServices().updateData(mutationPlan);
+            long firstUpsertSelectTime = System.currentTimeMillis() - startTime;
+
+            // for global indexes on non transactional tables we might have to
+            // run a second index population upsert select to handle data rows
+            // that were being written on the server while the index was created
+            long sleepTime =
+                    connection
+                            .getQueryServices()
+                            .getProps()
+                            .getLong(QueryServices.INDEX_POPULATION_SLEEP_TIME,
+                                QueryServicesOptions.DEFAULT_INDEX_POPULATION_SLEEP_TIME);
+            if (!dataTableRef.getTable().isTransactional() && sleepTime > 0) {
+                long delta = sleepTime - firstUpsertSelectTime;
+                if (delta > 0) {
+                    try {
+                        Thread.sleep(delta);
+                    } catch (InterruptedException e) {
+                        Thread.currentThread().interrupt();
+                        throw new SQLExceptionInfo.Builder(SQLExceptionCode.INTERRUPTED_EXCEPTION)
+                                .setRootCause(e).build().buildException();
+                    }
+                }
+                // set the min timestamp of second index upsert select some time before the index
+                // was created
+                long minTimestamp = index.getTimeStamp() - firstUpsertSelectTime;
+                try {
+                    mutationPlan.getContext().getScan().setTimeRange(minTimestamp, scn);
                 } catch (IOException e) {
                     throw new SQLException(e);
                 }
+                MutationState newMutationState =
+                        connection.getQueryServices().updateData(mutationPlan);
+                state.join(newMutationState);
             }
-            MutationState state = connection.getQueryServices().updateData(mutationPlan);
+            
             indexStatement = FACTORY.alterIndex(FACTORY.namedTable(null,
-                TableName.create(index.getSchemaName().getString(), index.getTableName().getString())),
-                dataTableRef.getTable().getTableName().getString(), false, PIndexState.ACTIVE);
+            		TableName.create(index.getSchemaName().getString(), index.getTableName().getString())),
+            		dataTableRef.getTable().getTableName().getString(), false, PIndexState.ACTIVE);
             alterIndex(indexStatement);
 
             return state;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a138cfe0/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumn.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumn.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumn.java
index 357ce6f..0f5fa44 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumn.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumn.java
@@ -58,4 +58,6 @@ public interface PColumn extends PDatum {
      * @return whether this column represents/stores the hbase cell timestamp.
      */
     boolean isRowTimestamp();
+    
+    boolean isDynamic();
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a138cfe0/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnImpl.java
index cff276b..a556f76 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnImpl.java
@@ -39,6 +39,7 @@ public class PColumnImpl implements PColumn {
     private boolean isViewReferenced;
     private String expressionStr;
     private boolean isRowTimestamp;
+    private boolean isDynamic;
     
     public PColumnImpl() {
     }
@@ -50,13 +51,13 @@ public class PColumnImpl implements PColumn {
                        Integer scale,
                        boolean nullable,
                        int position,
-                       SortOrder sortOrder, Integer arrSize, byte[] viewConstant, boolean isViewReferenced, String expressionStr, boolean isRowTimestamp) {
-        init(name, familyName, dataType, maxLength, scale, nullable, position, sortOrder, arrSize, viewConstant, isViewReferenced, expressionStr, isRowTimestamp);
+                       SortOrder sortOrder, Integer arrSize, byte[] viewConstant, boolean isViewReferenced, String expressionStr, boolean isRowTimestamp, boolean isDynamic) {
+        init(name, familyName, dataType, maxLength, scale, nullable, position, sortOrder, arrSize, viewConstant, isViewReferenced, expressionStr, isRowTimestamp, isDynamic);
     }
 
     public PColumnImpl(PColumn column, int position) {
         this(column.getName(), column.getFamilyName(), column.getDataType(), column.getMaxLength(),
-                column.getScale(), column.isNullable(), position, column.getSortOrder(), column.getArraySize(), column.getViewConstant(), column.isViewReferenced(), column.getExpressionStr(), column.isRowTimestamp());
+                column.getScale(), column.isNullable(), position, column.getSortOrder(), column.getArraySize(), column.getViewConstant(), column.isViewReferenced(), column.getExpressionStr(), column.isRowTimestamp(), column.isDynamic());
     }
 
     private void init(PName name,
@@ -68,7 +69,7 @@ public class PColumnImpl implements PColumn {
             int position,
             SortOrder sortOrder,
             Integer arrSize,
-            byte[] viewConstant, boolean isViewReferenced, String expressionStr, boolean isRowTimestamp) {
+            byte[] viewConstant, boolean isViewReferenced, String expressionStr, boolean isRowTimestamp, boolean isDynamic) {
     	Preconditions.checkNotNull(sortOrder);
         this.dataType = dataType;
         if (familyName == null) {
@@ -92,6 +93,7 @@ public class PColumnImpl implements PColumn {
         this.isViewReferenced = isViewReferenced;
         this.expressionStr = expressionStr;
         this.isRowTimestamp = isRowTimestamp;
+        this.isDynamic = isDynamic;
     }
 
     @Override
@@ -198,6 +200,11 @@ public class PColumnImpl implements PColumn {
     public boolean isRowTimestamp() {
         return isRowTimestamp;
     }
+    
+    @Override
+    public boolean isDynamic() {
+        return isDynamic;
+    }
 
     /**
      * Create a PColumn instance from PBed PColumn instance
@@ -240,8 +247,12 @@ public class PColumnImpl implements PColumn {
 	        expressionStr = column.getExpression();
         }
         boolean isRowTimestamp = column.getIsRowTimestamp();
+        boolean isDynamic = false;
+        if (column.hasIsDynamic()) {
+        	isDynamic = column.getIsDynamic();
+        }
         return new PColumnImpl(columnName, familyName, dataType, maxLength, scale, nullable, position, sortOrder,
-                arraySize, viewConstant, isViewReferenced, expressionStr, isRowTimestamp);
+                arraySize, viewConstant, isViewReferenced, expressionStr, isRowTimestamp, isDynamic);
     }
 
     public static PTableProtos.PColumn toProto(PColumn column) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a138cfe0/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
index 66b4af3..413d116 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
@@ -414,7 +414,7 @@ public class PMetaDataImpl implements PMetaData {
             // Update position of columns that follow removed column
             for (int i = position+1; i < oldColumns.size(); i++) {
                 PColumn oldColumn = oldColumns.get(i);
-                PColumn newColumn = new PColumnImpl(oldColumn.getName(), oldColumn.getFamilyName(), oldColumn.getDataType(), oldColumn.getMaxLength(), oldColumn.getScale(), oldColumn.isNullable(), i-1+positionOffset, oldColumn.getSortOrder(), oldColumn.getArraySize(), oldColumn.getViewConstant(), oldColumn.isViewReferenced(), null, oldColumn.isRowTimestamp());
+                PColumn newColumn = new PColumnImpl(oldColumn.getName(), oldColumn.getFamilyName(), oldColumn.getDataType(), oldColumn.getMaxLength(), oldColumn.getScale(), oldColumn.isNullable(), i-1+positionOffset, oldColumn.getSortOrder(), oldColumn.getArraySize(), oldColumn.getViewConstant(), oldColumn.isViewReferenced(), null, oldColumn.isRowTimestamp(), oldColumn.isDynamic());
                 columns.add(newColumn);
             }
             

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a138cfe0/phoenix-core/src/main/java/org/apache/phoenix/schema/SaltingUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/SaltingUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/SaltingUtil.java
index 4ac54cb..734a9ed 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/SaltingUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/SaltingUtil.java
@@ -38,7 +38,7 @@ public class SaltingUtil {
     public static final String SALTING_COLUMN_NAME = "_SALT";
     public static final String SALTED_ROW_KEY_NAME = "_SALTED_KEY";
     public static final PColumnImpl SALTING_COLUMN = new PColumnImpl(
-            PNameFactory.newName(SALTING_COLUMN_NAME), null, PBinary.INSTANCE, 1, 0, false, 0, SortOrder.getDefault(), 0, null, false, null, false);
+            PNameFactory.newName(SALTING_COLUMN_NAME), null, PBinary.INSTANCE, 1, 0, false, 0, SortOrder.getDefault(), 0, null, false, null, false, false);
     public static final RowKeySchema VAR_BINARY_SALTED_SCHEMA = new RowKeySchemaBuilder(2)
         .addField(SALTING_COLUMN, false, SortOrder.getDefault())
         .addField(SchemaUtil.VAR_BINARY_DATUM, false, SortOrder.getDefault()).build();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a138cfe0/phoenix-core/src/test/java/org/apache/phoenix/execute/CorrelatePlanTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/execute/CorrelatePlanTest.java b/phoenix-core/src/test/java/org/apache/phoenix/execute/CorrelatePlanTest.java
index 72f3e01..6b89187 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/execute/CorrelatePlanTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/execute/CorrelatePlanTest.java
@@ -226,7 +226,7 @@ public class CorrelatePlanTest {
             Expression expr = LiteralExpression.newConstant(row[i]);
             columns.add(new PColumnImpl(PNameFactory.newName(name), PNameFactory.newName(TupleProjector.VALUE_COLUMN_FAMILY),
                     expr.getDataType(), expr.getMaxLength(), expr.getScale(), expr.isNullable(),
-                    i, expr.getSortOrder(), null, null, false, name, false));
+                    i, expr.getSortOrder(), null, null, false, name, false, false));
         }
         try {
             PTable pTable = PTableImpl.makePTable(null, PName.EMPTY_NAME, PName.EMPTY_NAME,

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a138cfe0/phoenix-core/src/test/java/org/apache/phoenix/execute/UnnestArrayPlanTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/execute/UnnestArrayPlanTest.java b/phoenix-core/src/test/java/org/apache/phoenix/execute/UnnestArrayPlanTest.java
index d508707..8b2b096 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/execute/UnnestArrayPlanTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/execute/UnnestArrayPlanTest.java
@@ -116,8 +116,8 @@ public class UnnestArrayPlanTest {
         LiteralExpression dummy = LiteralExpression.newConstant(null, arrayType);
         RowKeyValueAccessor accessor = new RowKeyValueAccessor(Arrays.asList(dummy), 0);
         UnnestArrayPlan plan = new UnnestArrayPlan(subPlan, new RowKeyColumnExpression(dummy, accessor), withOrdinality);
-        PColumn elemColumn = new PColumnImpl(PNameFactory.newName("ELEM"), PNameFactory.newName(TupleProjector.VALUE_COLUMN_FAMILY), baseType, null, null, true, 0, SortOrder.getDefault(), null, null, false, "", false);
-        PColumn indexColumn = withOrdinality ? new PColumnImpl(PNameFactory.newName("IDX"), PNameFactory.newName(TupleProjector.VALUE_COLUMN_FAMILY), PInteger.INSTANCE, null, null, true, 0, SortOrder.getDefault(), null, null, false, "", false) : null;
+        PColumn elemColumn = new PColumnImpl(PNameFactory.newName("ELEM"), PNameFactory.newName(TupleProjector.VALUE_COLUMN_FAMILY), baseType, null, null, true, 0, SortOrder.getDefault(), null, null, false, "", false, false);
+        PColumn indexColumn = withOrdinality ? new PColumnImpl(PNameFactory.newName("IDX"), PNameFactory.newName(TupleProjector.VALUE_COLUMN_FAMILY), PInteger.INSTANCE, null, null, true, 0, SortOrder.getDefault(), null, null, false, "", false, false) : null;
         List<PColumn> columns = withOrdinality ? Arrays.asList(elemColumn, indexColumn) : Arrays.asList(elemColumn);
         ProjectedColumnExpression elemExpr = new ProjectedColumnExpression(elemColumn, columns, 0, elemColumn.getName().getString());
         ProjectedColumnExpression indexExpr = withOrdinality ? new ProjectedColumnExpression(indexColumn, columns, 1, indexColumn.getName().getString()) : null;


[23/50] [abbrv] phoenix git commit: PHOENIX-2613 Infinite loop in SkipScan when seeking past null value in non leading primary key column

Posted by ma...@apache.org.
PHOENIX-2613 Infinite loop in SkipScan when seeking past null value in non leading primary key column


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

Branch: refs/heads/calcite
Commit: 462509625853bf85d0fdeb33fe3cf36bf5c94bec
Parents: 6251f8f
Author: James Taylor <jt...@salesforce.com>
Authored: Thu Jan 21 17:58:20 2016 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Thu Jan 21 17:58:20 2016 -0800

----------------------------------------------------------------------
 .../apache/phoenix/end2end/SkipScanQueryIT.java | 22 +++++++
 .../apache/phoenix/filter/SkipScanFilter.java   | 24 ++++----
 .../phoenix/filter/SkipScanFilterTest.java      | 65 +++++++++++++-------
 3 files changed, 77 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/46250962/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipScanQueryIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipScanQueryIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipScanQueryIT.java
index 9f73550..31994bb 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipScanQueryIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipScanQueryIT.java
@@ -434,4 +434,26 @@ public class SkipScanQueryIT extends BaseHBaseManagedTimeIT {
             conn.close();
         }
     }
+    
+    @Test
+    public void testNullInfiniteLoop() throws Exception {
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            conn.setAutoCommit(true);
+            conn.createStatement().execute(
+              "create table NULL_TEST"+
+              "("+
+                     "CREATETIME VARCHAR,"+
+                     "ACCOUNTID VARCHAR,"+
+                     "SERVICENAME VARCHAR,"+
+                     "SPAN.APPID VARCHAR,"+
+                     "CONSTRAINT pk PRIMARY KEY(CREATETIME,ACCOUNTID,SERVICENAME)"+
+              ")");
+
+            conn.createStatement().execute("upsert into NULL_TEST(CREATETIME,SERVICENAME,SPAN.APPID) values('20160116141006','servlet','android')");
+            conn.createStatement().execute("upsert into NULL_TEST(CREATETIME,ACCOUNTID,SERVICENAME,SPAN.APPID) values('20160116151006','2404787','jdbc','ios')");
+            ResultSet rs = conn.createStatement().executeQuery("select * from NULL_TEST where CREATETIME>='20160116121006' and  CREATETIME<='20160116181006' and ACCOUNTID='2404787'");
+            assertTrue(rs.next());
+            assertFalse(rs.next());
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/46250962/phoenix-core/src/main/java/org/apache/phoenix/filter/SkipScanFilter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/filter/SkipScanFilter.java b/phoenix-core/src/main/java/org/apache/phoenix/filter/SkipScanFilter.java
index 77b4cf6..00320ce 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/filter/SkipScanFilter.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/filter/SkipScanFilter.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.io.Writable;
 import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.query.KeyRange.Bound;
 import org.apache.phoenix.schema.RowKeySchema;
+import org.apache.phoenix.schema.ValueSchema.Field;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.ScanUtil;
 import org.apache.phoenix.util.ScanUtil.BytesComparator;
@@ -161,8 +162,7 @@ public class SkipScanFilter extends FilterBase implements Writable {
                                 .getRowOffset(), previousCellHint.getRowLength()) > 0;
         if (!isHintAfterPrevious) {
             String msg = "The next hint must come after previous hint (prev=" + previousCellHint + ", next=" + nextCellHint + ", kv=" + kv + ")";
-            assert isHintAfterPrevious : msg;
-            logger.warn(msg);
+            throw new IllegalStateException(msg);
         }
     }
     
@@ -243,11 +243,6 @@ public class SkipScanFilter extends FilterBase implements Writable {
         }
         // Short circuit out if we only have a single set of keys
         if (slots.size() == 1) {
-//            int offset = slots.get(0).get(endPos).compareLowerToUpperBound(upperExclusiveKey) < 0 ? 1 : 0;
-//            if (endPos + offset <= startPos) {
-//                return false;
-//            }
-//            List<KeyRange> newRanges = slots.get(0).subList(startPos, endPos + offset);
             if (newSlots != null) {
                 List<KeyRange> newRanges = slots.get(0).subList(startPos, endPos+1);
                 newSlots.add(newRanges);
@@ -439,7 +434,7 @@ public class SkipScanFilter extends FilterBase implements Writable {
                     setStartKey();
                     schema.reposition(ptr, ScanUtil.getRowKeyPosition(slotSpan, i), ScanUtil.getRowKeyPosition(slotSpan, j), minOffset, maxOffset, slotSpan[j]);
                 } else {
-                    int currentLength = setStartKey(ptr, minOffset, j+1, nSlots);
+                    int currentLength = setStartKey(ptr, minOffset, j+1, nSlots, false);
                     // From here on, we use startKey as our buffer (resetting minOffset and maxOffset)
                     // We've copied the part of the current key above that we need into startKey
                     // Reinitialize the iterator to be positioned at previous slot position
@@ -454,7 +449,7 @@ public class SkipScanFilter extends FilterBase implements Writable {
             } else if (slots.get(i).get(position[i]).compareLowerToUpperBound(ptr, comparator) > 0) {
                 // Our current key is less than the lower range of the current position in the current slot.
                 // Seek to the lower range, since it's bigger than the current key
-                setStartKey(ptr, minOffset, i, nSlots);
+                setStartKey(ptr, minOffset, i, nSlots, false);
                 return ReturnCode.SEEK_NEXT_USING_HINT;
             } else { // We're in range, check the next slot
                 if (!slots.get(i).get(position[i]).isSingleKey() && i < earliestRangeIndex) {
@@ -477,7 +472,7 @@ public class SkipScanFilter extends FilterBase implements Writable {
                         break;
                     }
                     // Otherwise we seek to the next start key because we're before it now
-                    setStartKey(ptr, minOffset, i, nSlots);
+                    setStartKey(ptr, minOffset, i, nSlots, true);
                     return ReturnCode.SEEK_NEXT_USING_HINT;
                 }
             }
@@ -521,13 +516,16 @@ public class SkipScanFilter extends FilterBase implements Writable {
         startKeyLength = setKey(Bound.LOWER, startKey, 0, 0);
     }
 
-    private int setStartKey(ImmutableBytesWritable ptr, int offset, int i, int nSlots) {
+    private int setStartKey(ImmutableBytesWritable ptr, int offset, int i, int nSlots, boolean atEndOfKey) {
         int length = ptr.getOffset() - offset;
         startKey = copyKey(startKey, length + this.maxKeyLength, ptr.get(), offset, length);
         startKeyLength = length;
         // Add separator byte if we're at end of the key, since trailing separator bytes are stripped
-        if (ptr.getLength() == 0 && i > 0 && i-1 < nSlots && !schema.getField(i-1).getDataType().isFixedWidth()) {
-            startKey[startKeyLength++] = SchemaUtil.getSeparatorByte(schema.rowKeyOrderOptimizable(), ptr.getLength()==0, schema.getField(i-1));
+        if (atEndOfKey && i > 0 && i-1 < nSlots) {
+            Field field = schema.getField(i-1);
+            if (!field.getDataType().isFixedWidth()) {
+                startKey[startKeyLength++] = SchemaUtil.getSeparatorByte(schema.rowKeyOrderOptimizable(), true, field);
+            }
         }
         startKeyLength += setKey(Bound.LOWER, startKey, startKeyLength, i);
         return length;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/46250962/phoenix-core/src/test/java/org/apache/phoenix/filter/SkipScanFilterTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/filter/SkipScanFilterTest.java b/phoenix-core/src/test/java/org/apache/phoenix/filter/SkipScanFilterTest.java
index 4cb71ff..8ed0add 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/filter/SkipScanFilterTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/filter/SkipScanFilterTest.java
@@ -106,36 +106,59 @@ public class SkipScanFilterTest extends TestCase {
     public static Collection<Object> data() {
         List<Object> testCases = Lists.newArrayList();
         // Variable length tests
+//        testCases.addAll(
+//                foreach(new KeyRange[][]{{
+//                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true),
+//                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("e"), true, Bytes.toBytes("e"), true),
+//                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("f"), true, Bytes.toBytes("f"), true),
+//                },
+//                {
+//                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
+//                },
+//                {
+//                    KeyRange.EVERYTHING_RANGE,
+//                },
+//                {
+//                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
+//                }},
+//                new int[4],
+//                new Include(ByteUtil.concat(Bytes.toBytes("a"),QueryConstants.SEPARATOR_BYTE_ARRAY, 
+//                                            Bytes.toBytes("b"), QueryConstants.SEPARATOR_BYTE_ARRAY,
+//                                            QueryConstants.SEPARATOR_BYTE_ARRAY,
+//                                            Bytes.toBytes("1") ) ),
+//                new SeekNext(ByteUtil.concat(Bytes.toBytes("e.f"),QueryConstants.SEPARATOR_BYTE_ARRAY, 
+//                                             Bytes.toBytes("b"), QueryConstants.SEPARATOR_BYTE_ARRAY,
+//                                             QueryConstants.SEPARATOR_BYTE_ARRAY,
+//                                             Bytes.toBytes("1") ), 
+//                            ByteUtil.concat(Bytes.toBytes("f"),QueryConstants.SEPARATOR_BYTE_ARRAY, 
+//                                            Bytes.toBytes("b") )),
+//                new Include(ByteUtil.concat(Bytes.toBytes("f"),QueryConstants.SEPARATOR_BYTE_ARRAY, 
+//                                            Bytes.toBytes("b"), QueryConstants.SEPARATOR_BYTE_ARRAY,
+//                                            QueryConstants.SEPARATOR_BYTE_ARRAY,
+//                                            Bytes.toBytes("1") ) ) )
+//        );
         testCases.addAll(
                 foreach(new KeyRange[][]{{
-                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("a"), true, Bytes.toBytes("a"), true),
-                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("e"), true, Bytes.toBytes("e"), true),
-                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("f"), true, Bytes.toBytes("f"), true),
+                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("20160116121006"), true, Bytes.toBytes("20160116181006"), true),
                 },
                 {
-                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("b"), true, Bytes.toBytes("b"), true),
-                },
+                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("2404787"), true, Bytes.toBytes("2404787"), true),
+                }/*,
                 {
                     KeyRange.EVERYTHING_RANGE,
                 },
                 {
-                    PVarchar.INSTANCE.getKeyRange(Bytes.toBytes("1"), true, Bytes.toBytes("1"), true),
-                }},
+                    KeyRange.EVERYTHING_RANGE,
+                }*/},
                 new int[4],
-                new Include(ByteUtil.concat(Bytes.toBytes("a"),QueryConstants.SEPARATOR_BYTE_ARRAY, 
-                                            Bytes.toBytes("b"), QueryConstants.SEPARATOR_BYTE_ARRAY,
+                new SeekNext(ByteUtil.concat(Bytes.toBytes("20160116141006"),QueryConstants.SEPARATOR_BYTE_ARRAY, 
                                             QueryConstants.SEPARATOR_BYTE_ARRAY,
-                                            Bytes.toBytes("1") ) ),
-                new SeekNext(ByteUtil.concat(Bytes.toBytes("e.f"),QueryConstants.SEPARATOR_BYTE_ARRAY, 
-                                             Bytes.toBytes("b"), QueryConstants.SEPARATOR_BYTE_ARRAY,
-                                             QueryConstants.SEPARATOR_BYTE_ARRAY,
-                                             Bytes.toBytes("1") ), 
-                            ByteUtil.concat(Bytes.toBytes("f"),QueryConstants.SEPARATOR_BYTE_ARRAY, 
-                                            Bytes.toBytes("b") )),
-                new Include(ByteUtil.concat(Bytes.toBytes("f"),QueryConstants.SEPARATOR_BYTE_ARRAY, 
-                                            Bytes.toBytes("b"), QueryConstants.SEPARATOR_BYTE_ARRAY,
-                                            QueryConstants.SEPARATOR_BYTE_ARRAY,
-                                            Bytes.toBytes("1") ) ) )
+                                            Bytes.toBytes("servlet") ),
+                             ByteUtil.concat(Bytes.toBytes("20160116141006"),QueryConstants.SEPARATOR_BYTE_ARRAY, 
+                                             Bytes.toBytes("2404787") )),
+                new Include(ByteUtil.concat(Bytes.toBytes("20160116151006"),QueryConstants.SEPARATOR_BYTE_ARRAY, 
+                                            Bytes.toBytes("2404787"), QueryConstants.SEPARATOR_BYTE_ARRAY,
+                                            Bytes.toBytes("jdbc"), QueryConstants.SEPARATOR_BYTE_ARRAY ) ) )
         );
         // Fixed length tests
         testCases.addAll(
@@ -342,7 +365,7 @@ public class SkipScanFilterTest extends TestCase {
         );
         return testCases;
     }
-
+    
     private static Collection<?> foreach(KeyRange[][] ranges, int[] widths, Expectation... expectations) {
         List<List<KeyRange>> cnf = Lists.transform(Lists.newArrayList(ranges), ARRAY_TO_LIST);
         List<Object> ret = Lists.newArrayList();


[43/50] [abbrv] phoenix git commit: PHOENIX-2543 Modify StatsCollectorIT to test transactional test cases

Posted by ma...@apache.org.
PHOENIX-2543 Modify StatsCollectorIT to test transactional test cases


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

Branch: refs/heads/calcite
Commit: 791a27cfd141af969bc3e2e25ac9c14bd1b17a90
Parents: ee63702
Author: Thomas D'Silva <td...@salesforce.com>
Authored: Thu Jan 28 13:26:40 2016 -0800
Committer: Thomas D'Silva <td...@salesforce.com>
Committed: Thu Jan 28 14:14:33 2016 -0800

----------------------------------------------------------------------
 .../phoenix/end2end/StatsCollectorIT.java       |  73 ++++++----
 .../org/apache/phoenix/tx/TransactionIT.java    | 137 +++++++++++++++----
 2 files changed, 152 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/791a27cf/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java
index 706ae4a..caba259 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java
@@ -30,6 +30,8 @@ import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
@@ -43,14 +45,23 @@ import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.TestUtil;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
 
 import com.google.common.collect.Maps;
 
-
+@RunWith(Parameterized.class)
 public class StatsCollectorIT extends StatsCollectorAbstractIT {
     private static final String STATS_TEST_TABLE_NAME = "S";
+    
+    private final String tableDDLOptions;
+    private final String tableName;
+    private final String fullTableName;
         
     @BeforeClass
     public static void doSetup() throws Exception {
@@ -62,30 +73,31 @@ public class StatsCollectorIT extends StatsCollectorAbstractIT {
         props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
         setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
     }
-
-    @Test
-    public void testUpdateStatsForNonTxnTable() throws Throwable {
-        helpTestUpdateStats(false);
+    
+    public StatsCollectorIT( boolean transactional) {
+        this.tableDDLOptions= transactional ? " TRANSACTIONAL=true" : "";
+        this.tableName = TestUtil.DEFAULT_DATA_TABLE_NAME + ( transactional ?  "_TXN" : "");
+        this.fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
     }
     
-    @Test
-    public void testUpdateStatsForTxnTable() throws Throwable {
-        helpTestUpdateStats(true);
+    @Parameters(name="transactional = {0}")
+    public static Collection<Boolean> data() {
+        return Arrays.asList(false,true);
     }
 
-	private void helpTestUpdateStats(boolean transactional) throws SQLException, IOException,
+    @Test
+    public void testUpdateStats() throws SQLException, IOException,
 			InterruptedException {
 		Connection conn;
         PreparedStatement stmt;
         ResultSet rs;
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        String tableName = "T" + (transactional ? "_TXN" : "");
         // props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 10));
         conn = DriverManager.getConnection(getUrl(), props);
         conn.createStatement().execute(
-                "CREATE TABLE " + tableName +" ( k VARCHAR, a_string_array VARCHAR(100) ARRAY[4], b_string_array VARCHAR(100) ARRAY[4] \n"
-                        + " CONSTRAINT pk PRIMARY KEY (k, b_string_array DESC))" 
-                		+ (transactional ? " TRANSACTIONAL=true" : ""));
+                "CREATE TABLE " + fullTableName +" ( k VARCHAR, a_string_array VARCHAR(100) ARRAY[4], b_string_array VARCHAR(100) ARRAY[4] \n"
+                        + " CONSTRAINT pk PRIMARY KEY (k, b_string_array DESC))"
+                		+ tableDDLOptions );
         String[] s;
         Array array;
         conn = upsertValues(props, tableName);
@@ -119,14 +131,14 @@ public class StatsCollectorIT extends StatsCollectorAbstractIT {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         conn = DriverManager.getConnection(getUrl(), props);
         conn.createStatement()
-                .execute("CREATE TABLE x ( k VARCHAR, c1.a bigint,c2.b bigint CONSTRAINT pk PRIMARY KEY (k)) \n");
-        conn.createStatement().execute("upsert into x values ('abc',1,3)");
-        conn.createStatement().execute("upsert into x values ('def',2,4)");
+                .execute("CREATE TABLE " + fullTableName +" ( k VARCHAR, c1.a bigint,c2.b bigint CONSTRAINT pk PRIMARY KEY (k))" + tableDDLOptions );
+        conn.createStatement().execute("upsert into " + fullTableName +" values ('abc',1,3)");
+        conn.createStatement().execute("upsert into " + fullTableName +" values ('def',2,4)");
         conn.commit();
         // CAll the update statistics query here
-        stmt = conn.prepareStatement("UPDATE STATISTICS X");
+        stmt = conn.prepareStatement("UPDATE STATISTICS " + fullTableName);
         stmt.execute();
-        rs = conn.createStatement().executeQuery("SELECT k FROM x");
+        rs = conn.createStatement().executeQuery("SELECT k FROM " + fullTableName);
         assertTrue(rs.next());
         assertEquals("abc", rs.getString(1));
         assertTrue(rs.next());
@@ -137,6 +149,7 @@ public class StatsCollectorIT extends StatsCollectorAbstractIT {
 
     @Test
     public void testUpdateStatsWithMultipleTables() throws Throwable {
+        String fullTableName2 = fullTableName+"_2";
         Connection conn;
         PreparedStatement stmt;
         ResultSet rs;
@@ -144,21 +157,21 @@ public class StatsCollectorIT extends StatsCollectorAbstractIT {
         // props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 10));
         conn = DriverManager.getConnection(getUrl(), props);
         conn.createStatement().execute(
-                "CREATE TABLE x ( k VARCHAR, a_string_array VARCHAR(100) ARRAY[4], b_string_array VARCHAR(100) ARRAY[4] \n"
-                        + " CONSTRAINT pk PRIMARY KEY (k, b_string_array DESC)) \n");
+                "CREATE TABLE " + fullTableName +" ( k VARCHAR, a_string_array VARCHAR(100) ARRAY[4], b_string_array VARCHAR(100) ARRAY[4] \n"
+                        + " CONSTRAINT pk PRIMARY KEY (k, b_string_array DESC))" + tableDDLOptions );
         conn.createStatement().execute(
-                "CREATE TABLE z ( k VARCHAR, a_string_array VARCHAR(100) ARRAY[4], b_string_array VARCHAR(100) ARRAY[4] \n"
-                        + " CONSTRAINT pk PRIMARY KEY (k, b_string_array DESC)) \n");
+                "CREATE TABLE " + fullTableName2 +" ( k VARCHAR, a_string_array VARCHAR(100) ARRAY[4], b_string_array VARCHAR(100) ARRAY[4] \n"
+                        + " CONSTRAINT pk PRIMARY KEY (k, b_string_array DESC))" + tableDDLOptions );
         String[] s;
         Array array;
-        conn = upsertValues(props, "x");
-        conn = upsertValues(props, "z");
+        conn = upsertValues(props, fullTableName);
+        conn = upsertValues(props, fullTableName2);
         // CAll the update statistics query here
-        stmt = conn.prepareStatement("UPDATE STATISTICS X");
+        stmt = conn.prepareStatement("UPDATE STATISTICS "+fullTableName);
         stmt.execute();
-        stmt = conn.prepareStatement("UPDATE STATISTICS Z");
+        stmt = conn.prepareStatement("UPDATE STATISTICS "+fullTableName2);
         stmt.execute();
-        stmt = upsertStmt(conn, "x");
+        stmt = upsertStmt(conn, fullTableName);
         stmt.setString(1, "z");
         s = new String[] { "xyz", "def", "ghi", "jkll", null, null, "xxx" };
         array = conn.createArrayOf("VARCHAR", s);
@@ -167,7 +180,7 @@ public class StatsCollectorIT extends StatsCollectorAbstractIT {
         array = conn.createArrayOf("VARCHAR", s);
         stmt.setArray(3, array);
         stmt.execute();
-        stmt = upsertStmt(conn, "z");
+        stmt = upsertStmt(conn, fullTableName2);
         stmt.setString(1, "z");
         s = new String[] { "xyz", "def", "ghi", "jkll", null, null, "xxx" };
         array = conn.createArrayOf("VARCHAR", s);
@@ -179,9 +192,9 @@ public class StatsCollectorIT extends StatsCollectorAbstractIT {
         conn.close();
         conn = DriverManager.getConnection(getUrl(), props);
         // This analyze would not work
-        stmt = conn.prepareStatement("UPDATE STATISTICS Z");
+        stmt = conn.prepareStatement("UPDATE STATISTICS "+fullTableName2);
         stmt.execute();
-        rs = conn.createStatement().executeQuery("SELECT k FROM Z");
+        rs = conn.createStatement().executeQuery("SELECT k FROM "+fullTableName2);
         assertTrue(rs.next());
         conn.close();
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/791a27cf/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
index 2794c47..b65b856 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
@@ -20,6 +20,8 @@ package org.apache.phoenix.tx;
 import static org.apache.phoenix.util.TestUtil.INDEX_DATA_SCHEMA;
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
 import static org.apache.phoenix.util.TestUtil.TRANSACTIONAL_DATA_TABLE;
+import static org.apache.phoenix.util.TestUtil.analyzeTable;
+import static org.apache.phoenix.util.TestUtil.getAllSplits;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -36,12 +38,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 
-import co.cask.tephra.TransactionContext;
-import co.cask.tephra.TransactionSystemClient;
-import co.cask.tephra.TxConstants;
-import co.cask.tephra.hbase11.TransactionAwareHTable;
-import co.cask.tephra.hbase11.coprocessor.TransactionProcessor;
-
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
@@ -56,6 +52,7 @@ import org.apache.phoenix.end2end.Shadower;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
@@ -71,6 +68,12 @@ import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
 
+import co.cask.tephra.TransactionContext;
+import co.cask.tephra.TransactionSystemClient;
+import co.cask.tephra.TxConstants;
+import co.cask.tephra.hbase11.TransactionAwareHTable;
+import co.cask.tephra.hbase11.coprocessor.TransactionProcessor;
+
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
@@ -86,8 +89,9 @@ public class TransactionIT extends BaseHBaseManagedTimeIT {
     @BeforeClass
     @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
     public static void doSetup() throws Exception {
-        Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
+        Map<String,String> props = Maps.newHashMapWithExpectedSize(2);
         props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
+        props.put(QueryServices.STATS_GUIDEPOST_WIDTH_BYTES_ATTRIB, Integer.toString(20));
         setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
     }
         
@@ -581,30 +585,107 @@ public class TransactionIT extends BaseHBaseManagedTimeIT {
     }
     
     @Test
-    public void testReadOnlyView() throws Exception {
-        Connection conn = DriverManager.getConnection(getUrl());
-        String ddl = "CREATE TABLE t (k INTEGER NOT NULL PRIMARY KEY, v1 DATE) TRANSACTIONAL=true";
-        conn.createStatement().execute(ddl);
-        ddl = "CREATE VIEW v (v2 VARCHAR) AS SELECT * FROM t where k>4";
-        conn.createStatement().execute(ddl);
-        for (int i = 0; i < 10; i++) {
-            conn.createStatement().execute("UPSERT INTO t VALUES(" + i + ")");
+    public void testReadOnlyViewWithStats() throws Exception {
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String ddl = "CREATE TABLE t (k INTEGER NOT NULL PRIMARY KEY, v1 DATE) TRANSACTIONAL=true";
+            conn.createStatement().execute(ddl);
+            ddl = "CREATE VIEW v (v2 VARCHAR) AS SELECT * FROM t where k>5";
+            conn.createStatement().execute(ddl);
+            for (int i = 0; i < 10; i++) {
+                conn.createStatement().execute("UPSERT INTO t VALUES(" + i + ")");
+            }
+            conn.commit();
+            
+            // verify rows are visible for stats
+            analyzeTable(conn, "v", true);
+            List<KeyRange> splits = getAllSplits(conn, "v");
+            assertEquals(4, splits.size());
+            
+            int count = 0;
+            ResultSet rs = conn.createStatement().executeQuery("SELECT k FROM t");
+            while (rs.next()) {
+                assertEquals(count++, rs.getInt(1));
+            }
+            assertEquals(10, count);
+            
+            count = 0;
+            rs = conn.createStatement().executeQuery("SELECT k FROM v");
+            while (rs.next()) {
+                assertEquals(6+count++, rs.getInt(1));
+            }
+            assertEquals(4, count);
         }
-        conn.commit();
-        
-        int count = 0;
-        ResultSet rs = conn.createStatement().executeQuery("SELECT k FROM t");
-        while (rs.next()) {
-            assertEquals(count++, rs.getInt(1));
+    }
+    
+    @Test
+    public void testReadOwnWritesWithStats() throws Exception {
+        try (Connection conn1 = DriverManager.getConnection(getUrl()); 
+                Connection conn2 = DriverManager.getConnection(getUrl())) {
+            String ddl = "CREATE TABLE t (k INTEGER NOT NULL PRIMARY KEY, v1 DATE) TRANSACTIONAL=true";
+            conn1.createStatement().execute(ddl);
+            ddl = "CREATE VIEW v (v2 VARCHAR) AS SELECT * FROM t where k>5";
+            conn1.createStatement().execute(ddl);
+            for (int i = 0; i < 10; i++) {
+                conn1.createStatement().execute("UPSERT INTO t VALUES(" + i + ")");
+            }
+    
+            // verify you can read your own writes
+            int count = 0;
+            ResultSet rs = conn1.createStatement().executeQuery("SELECT k FROM t");
+            while (rs.next()) {
+                assertEquals(count++, rs.getInt(1));
+            }
+            assertEquals(10, count);
+            
+            count = 0;
+            rs = conn1.createStatement().executeQuery("SELECT k FROM v");
+            while (rs.next()) {
+                assertEquals(6+count++, rs.getInt(1));
+            }
+            assertEquals(4, count);
+            
+            // verify stats can see the read own writes rows
+            analyzeTable(conn2, "v", true);
+            List<KeyRange> splits = getAllSplits(conn2, "v");
+            assertEquals(4, splits.size());
         }
-        assertEquals(10, count);
-        
-        count = 0;
-        rs = conn.createStatement().executeQuery("SELECT k FROM v");
-        while (rs.next()) {
-            assertEquals(5+count++, rs.getInt(1));
+    }
+    
+    @Test
+    public void testInvalidRowsWithStats() throws Exception {
+        try (Connection conn1 = DriverManager.getConnection(getUrl()); 
+                Connection conn2 = DriverManager.getConnection(getUrl())) {
+            String ddl = "CREATE TABLE t (k INTEGER NOT NULL PRIMARY KEY, v1 DATE) TRANSACTIONAL=true";
+            conn1.createStatement().execute(ddl);
+            ddl = "CREATE VIEW v (v2 VARCHAR) AS SELECT * FROM t where k>5";
+            conn1.createStatement().execute(ddl);
+            for (int i = 0; i < 10; i++) {
+                conn1.createStatement().execute("UPSERT INTO t VALUES(" + i + ")");
+            }
+    
+            // verify you can read your own writes
+            int count = 0;
+            ResultSet rs = conn1.createStatement().executeQuery("SELECT k FROM t");
+            while (rs.next()) {
+                assertEquals(count++, rs.getInt(1));
+            }
+            assertEquals(10, count);
+            
+            count = 0;
+            rs = conn1.createStatement().executeQuery("SELECT k FROM v");
+            while (rs.next()) {
+                assertEquals(6+count++, rs.getInt(1));
+            }
+            assertEquals(4, count);
+            
+            Thread.sleep(DEFAULT_TXN_TIMEOUT_SECONDS*1000+20000);
+            assertEquals("There should be one invalid transaction", 1, txManager.getInvalidSize());
+            
+            // verify stats can see the rows from the invalid transaction
+            analyzeTable(conn2, "v", true);
+            List<KeyRange> splits = getAllSplits(conn2, "v");
+            assertEquals(4, splits.size());
         }
-        assertEquals(5, count);
     }
     
     @Test


[37/50] [abbrv] phoenix git commit: PHOENIX-1854 Investigate not setting ProjectedColumnExpression in TupleProjector expressions array

Posted by ma...@apache.org.
PHOENIX-1854 Investigate not setting ProjectedColumnExpression in TupleProjector expressions array


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

Branch: refs/heads/calcite
Commit: de4f174272be4fddcc8bf6dada048b4e70394f52
Parents: e2b6009
Author: maryannxue <ma...@gmail.com>
Authored: Tue Jan 26 20:27:10 2016 -0500
Committer: maryannxue <ma...@gmail.com>
Committed: Tue Jan 26 20:27:10 2016 -0500

----------------------------------------------------------------------
 .../org/apache/phoenix/compile/OrderPreservingTracker.java     | 6 ++----
 1 file changed, 2 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/de4f1742/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderPreservingTracker.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderPreservingTracker.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderPreservingTracker.java
index 65245f3..7715747 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderPreservingTracker.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderPreservingTracker.java
@@ -216,10 +216,8 @@ public class OrderPreservingTracker {
                 return super.visit(node);
             }
             Expression expression = projector.getExpressions()[node.getPosition()];
-            // FIXME: prevents infinite recursion for union all in subquery, but
-            // should a ProjectedColumnExpression be used in this case? Wouldn't
-            // it make more sense to not create this wrapper in this case?
-            if (expression == node) {
+            // Only look one level down the projection.
+            if (expression instanceof ProjectedColumnExpression) {
                 return super.visit(node);
             }
             return expression.accept(this);


[38/50] [abbrv] phoenix git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/phoenix

Posted by ma...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/phoenix


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

Branch: refs/heads/calcite
Commit: aaab87a88f162384cbd94ab1703f7fc5daec5751
Parents: de4f174 bd06dfb
Author: maryannxue <ma...@gmail.com>
Authored: Tue Jan 26 20:27:17 2016 -0500
Committer: maryannxue <ma...@gmail.com>
Committed: Tue Jan 26 20:27:17 2016 -0500

----------------------------------------------------------------------
 .../query/ConnectionQueryServicesImpl.java      | 82 ++++----------------
 .../phoenix/query/QueryServicesOptions.java     |  2 +-
 2 files changed, 14 insertions(+), 70 deletions(-)
----------------------------------------------------------------------



[19/50] [abbrv] phoenix git commit: PHOENIX-2616 Indexes over immutable tables not marked as immutable

Posted by ma...@apache.org.
PHOENIX-2616 Indexes over immutable tables not marked as immutable


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

Branch: refs/heads/calcite
Commit: 7795f80fa67483e31ab9b845013b8b332a7e5e05
Parents: c966935
Author: James Taylor <ja...@apache.org>
Authored: Wed Jan 20 19:09:19 2016 -0800
Committer: James Taylor <ja...@apache.org>
Committed: Wed Jan 20 19:09:19 2016 -0800

----------------------------------------------------------------------
 .../apache/phoenix/query/ConnectionQueryServicesImpl.java    | 8 +++-----
 1 file changed, 3 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/7795f80f/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 73999b8..e8c5e1a 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
@@ -2427,6 +2427,9 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
 									// Drop old stats table so that new stats table is created
 									metaConnection = dropStatsTable(metaConnection,
 											MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0);
+									// Clear the server cache so the above changes make it over to any clients
+									// that already have cached data.
+									clearCache();
                                 }
                                 
                             }
@@ -2608,11 +2611,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
 				throw sqlE;
 			}
 		}
-		metaConnection.removeTable(null, PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME,
-				PhoenixDatabaseMetaData.SYSTEM_STATS_TABLE, MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP);
-		clearTableFromCache(ByteUtil.EMPTY_BYTE_ARRAY, PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME_BYTES,
-				PhoenixDatabaseMetaData.SYSTEM_STATS_TABLE_BYTES, MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP);
-		clearTableRegionCache(PhoenixDatabaseMetaData.SYSTEM_STATS_NAME_BYTES);
 		return metaConnection;
 	}
 


[14/50] [abbrv] phoenix git commit: PHOENIX-2417 Compress memory used by row key byte[] of guideposts (Ankit Singhal)

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/6a27cecc/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
----------------------------------------------------------------------
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 1611466..f5c9295 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
@@ -75,6 +75,7 @@ import com.google.common.collect.ImmutableSortedMap;
 import com.google.common.collect.ListMultimap;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
+import com.google.protobuf.HBaseZeroCopyByteString;
 import com.sun.istack.NotNull;
 
 import co.cask.tephra.TxConstants;
@@ -1016,27 +1017,17 @@ public class PTableImpl implements PTable {
 
       boolean isImmutableRows = table.getIsImmutableRows();
       SortedMap<byte[], GuidePostsInfo> tableGuidePosts = new TreeMap<byte[], GuidePostsInfo>(Bytes.BYTES_COMPARATOR);
-        for (PTableProtos.PTableStats pTableStatsProto : table.getGuidePostsList()) {
-            List<byte[]> value = Lists.newArrayListWithExpectedSize(pTableStatsProto.getValuesCount());
-            for (int j = 0; j < pTableStatsProto.getValuesCount(); j++) {
-                value.add(pTableStatsProto.getValues(j).toByteArray());
-            }
-            // No op
-            pTableStatsProto.getGuidePostsByteCount();
-            value = Lists.newArrayListWithExpectedSize(pTableStatsProto.getValuesCount());
-            PGuidePosts pGuidePosts = pTableStatsProto.getPGuidePosts();
-            for(int j = 0; j < pGuidePosts.getGuidePostsCount(); j++) {
-                value.add(pGuidePosts.getGuidePosts(j).toByteArray());
-            }
-            long guidePostsByteCount = pGuidePosts.getByteCount();
-            long rowCount = pGuidePosts.getRowCount();
-            // TODO : Not exposing MIN/MAX key outside to client 
-            GuidePostsInfo info =
-                    new GuidePostsInfo(guidePostsByteCount, value, rowCount);
-            tableGuidePosts.put(pTableStatsProto.getKey().toByteArray(), info);
+      for (PTableProtos.PTableStats pTableStatsProto : table.getGuidePostsList()) {
+          PGuidePosts pGuidePosts = pTableStatsProto.getPGuidePosts();
+          long guidePostsByteCount = pGuidePosts.getByteCount();
+          long rowCount = pGuidePosts.getRowCount();
+          int maxLength = pGuidePosts.getMaxLength();
+          int guidePostsCount = pGuidePosts.getEncodedGuidePostsCount();
+          GuidePostsInfo info = new GuidePostsInfo(guidePostsByteCount,
+                  new ImmutableBytesWritable(HBaseZeroCopyByteString.zeroCopyGetBytes(pGuidePosts.getEncodedGuidePosts())), rowCount, maxLength, guidePostsCount);
+          tableGuidePosts.put(pTableStatsProto.getKey().toByteArray(), info);
       }
       PTableStats stats = new PTableStatsImpl(tableGuidePosts, table.getStatsTimeStamp());
-
       PName dataTableName = null;
       if (table.hasDataTableNameBytes()) {
         dataTableName = PNameFactory.newName(table.getDataTableNameBytes().toByteArray());
@@ -1141,16 +1132,14 @@ public class PTableImpl implements PTable {
       for (Map.Entry<byte[], GuidePostsInfo> entry : table.getTableStats().getGuidePosts().entrySet()) {
          PTableProtos.PTableStats.Builder statsBuilder = PTableProtos.PTableStats.newBuilder();
          statsBuilder.setKey(ByteStringer.wrap(entry.getKey()));
-         for (byte[] stat : entry.getValue().getGuidePosts()) {
-             statsBuilder.addValues(ByteStringer.wrap(stat));
-         }
          statsBuilder.setGuidePostsByteCount(entry.getValue().getByteCount());
+         statsBuilder.setGuidePostsCount(entry.getValue().getGuidePostsCount());
          PGuidePostsProtos.PGuidePosts.Builder guidePstsBuilder = PGuidePostsProtos.PGuidePosts.newBuilder();
-         for (byte[] stat : entry.getValue().getGuidePosts()) {
-             guidePstsBuilder.addGuidePosts(ByteStringer.wrap(stat));
-         }
+         guidePstsBuilder.setEncodedGuidePosts(ByteStringer.wrap(entry.getValue().getGuidePosts().get()));
          guidePstsBuilder.setByteCount(entry.getValue().getByteCount());
          guidePstsBuilder.setRowCount(entry.getValue().getRowCount());
+         guidePstsBuilder.setMaxLength(entry.getValue().getMaxLength());
+         guidePstsBuilder.setEncodedGuidePostsCount(entry.getValue().getGuidePostsCount());
          statsBuilder.setPGuidePosts(guidePstsBuilder);
          builder.addGuidePosts(statsBuilder.build());
       }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6a27cecc/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/GuidePostsInfo.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/GuidePostsInfo.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/GuidePostsInfo.java
index 0f1dbeb..da7d3a5 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/GuidePostsInfo.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/GuidePostsInfo.java
@@ -17,14 +17,8 @@
  */
 package org.apache.phoenix.schema.stats;
 
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.util.Bytes;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.util.ByteUtil;
 /**
  *  A class that holds the guidePosts of a region and also allows combining the 
  *  guidePosts of different regions when the GuidePostsInfo is formed for a table.
@@ -34,7 +28,7 @@ public class GuidePostsInfo {
     /**
      * the total number of guidePosts for the table combining all the guidePosts per region per cf.
      */
-    private List<byte[]> guidePosts;
+    private ImmutableBytesWritable guidePosts;
     /**
      * The bytecount that is flattened across the total number of guide posts.
      */
@@ -45,7 +39,19 @@ public class GuidePostsInfo {
      */
     private long rowCount = 0;
     
-    private long keyByteSize; // Total number of bytes in keys stored in guidePosts
+    /**
+     * Maximum length of a guidePost collected
+     */
+    private int maxLength;
+    
+    public final static GuidePostsInfo EMPTY_GUIDEPOST = new GuidePostsInfo(0,
+            new ImmutableBytesWritable(ByteUtil.EMPTY_BYTE_ARRAY), 0, 0, 0);
+
+    public int getMaxLength() {
+        return maxLength;
+    }
+    
+    private int guidePostsCount;
 
     /**
      * Constructor that creates GuidePostsInfo per region
@@ -53,22 +59,20 @@ public class GuidePostsInfo {
      * @param guidePosts
      * @param rowCount
      */
-    public GuidePostsInfo(long byteCount, List<byte[]> guidePosts, long rowCount) {
-        this.guidePosts = ImmutableList.copyOf(guidePosts);
-        int size = 0;
-        for (byte[] key : guidePosts) {
-            size += key.length;
-        }
-        this.keyByteSize = size;
+    public GuidePostsInfo(long byteCount, ImmutableBytesWritable guidePosts, long rowCount, int maxLength, int guidePostsCount) {
+        this.guidePosts = new ImmutableBytesWritable(guidePosts);
+        this.maxLength = maxLength;
         this.byteCount = byteCount;
         this.rowCount = rowCount;
+        this.guidePostsCount = guidePostsCount;
     }
     
+    
     public long getByteCount() {
         return byteCount;
     }
 
-    public List<byte[]> getGuidePosts() {
+    public ImmutableBytesWritable getGuidePosts() {
         return guidePosts;
     }
 
@@ -76,70 +80,8 @@ public class GuidePostsInfo {
         return this.rowCount;
     }
     
-    public void incrementRowCount() {
-        this.rowCount++;
-    }
-
-    /**
-     * Combines the GuidePosts per region into one.
-     * @param oldInfo
-     */
-    public void combine(GuidePostsInfo oldInfo) {
-        if (!oldInfo.getGuidePosts().isEmpty()) {
-            byte[] newFirstKey = oldInfo.getGuidePosts().get(0);
-            byte[] existingLastKey;
-            if (!this.getGuidePosts().isEmpty()) {
-                existingLastKey = this.getGuidePosts().get(this.getGuidePosts().size() - 1);
-            } else {
-                existingLastKey = HConstants.EMPTY_BYTE_ARRAY;
-            }
-            int size = oldInfo.getGuidePosts().size();
-            // If the existing guidePosts is lesser than the new RegionInfo that we are combining
-            // then add the new Region info to the end of the current GuidePosts.
-            // If the new region info is smaller than the existing guideposts then add the existing
-            // guide posts after the new guideposts.
-            List<byte[]> newTotalGuidePosts = new ArrayList<byte[]>(this.getGuidePosts().size() + size);
-            if (Bytes.compareTo(existingLastKey, newFirstKey) <= 0) {
-                newTotalGuidePosts.addAll(this.getGuidePosts());
-                newTotalGuidePosts.addAll(oldInfo.getGuidePosts());
-            } else {
-                newTotalGuidePosts.addAll(oldInfo.getGuidePosts());
-                newTotalGuidePosts.addAll(this.getGuidePosts());
-            }
-            this.guidePosts = ImmutableList.copyOf(newTotalGuidePosts);
-        }
-        this.byteCount += oldInfo.getByteCount();
-        this.keyByteSize += oldInfo.keyByteSize;
-        this.rowCount += oldInfo.getRowCount();
-    }
-    
-    /**
-     * The guide posts, rowCount and byteCount are accumulated every time a guidePosts depth is
-     * reached while collecting stats.
-     * @param row
-     * @param byteCount
-     * @return
-     */
-    public boolean addGuidePost(byte[] row, long byteCount, long rowCount) {
-        if (guidePosts.isEmpty() || Bytes.compareTo(row, guidePosts.get(guidePosts.size() - 1)) > 0) {
-            List<byte[]> newGuidePosts = Lists.newArrayListWithExpectedSize(this.getGuidePosts().size() + 1);
-            newGuidePosts.addAll(guidePosts);
-            newGuidePosts.add(row);
-            this.guidePosts = ImmutableList.copyOf(newGuidePosts);
-            this.byteCount += byteCount;
-            this.keyByteSize += row.length;
-            this.rowCount+=rowCount;
-            return true;
-        }
-        return false;
-    }
-    
-    public boolean addGuidePost(byte[] row) {
-        return addGuidePost(row, 0, 0);
-    }
-
-    public boolean addGuidePost(byte[] row, long byteCount) {
-        return addGuidePost(row, byteCount, 0);
+    public int getGuidePostsCount() {
+        return guidePostsCount;
     }
 
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6a27cecc/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/GuidePostsInfoBuilder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/GuidePostsInfoBuilder.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/GuidePostsInfoBuilder.java
new file mode 100644
index 0000000..f3ada82
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/GuidePostsInfoBuilder.java
@@ -0,0 +1,113 @@
+/*
+ * 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.phoenix.schema.stats;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.PrefixByteCodec;
+import org.apache.phoenix.util.PrefixByteEncoder;
+import org.apache.phoenix.util.TrustedByteArrayOutputStream;
+
+/*
+ * Builder to help in adding guidePosts and building guidePostInfo. This is used when we are collecting stats or reading stats for a table.
+ */
+
+public class GuidePostsInfoBuilder {
+    private PrefixByteEncoder encoder;
+    private byte[] lastRow;
+    private ImmutableBytesWritable guidePosts=new ImmutableBytesWritable(ByteUtil.EMPTY_BYTE_ARRAY);
+    private long byteCount = 0;
+    private int guidePostsCount;
+    
+    /**
+     * The rowCount that is flattened across the total number of guide posts.
+     */
+    private long rowCount = 0;
+    
+    /**
+     * Maximum length of a guidePost collected
+     */
+    private int maxLength;
+    private DataOutputStream output;
+    private TrustedByteArrayOutputStream stream;
+    
+    public final static GuidePostsInfo EMPTY_GUIDEPOST = new GuidePostsInfo(0,
+            new ImmutableBytesWritable(ByteUtil.EMPTY_BYTE_ARRAY), 0, 0, 0);
+
+    public int getMaxLength() {
+        return maxLength;
+    }
+    public GuidePostsInfoBuilder(){
+        this.stream = new TrustedByteArrayOutputStream(1);
+        this.output = new DataOutputStream(stream);
+        this.encoder=new PrefixByteEncoder();
+        lastRow = ByteUtil.EMPTY_BYTE_ARRAY;
+    }
+
+    /**
+     * The guide posts, rowCount and byteCount are accumulated every time a guidePosts depth is
+     * reached while collecting stats.
+     * @param row
+     * @param byteCount
+     * @return
+     * @throws IOException 
+     */
+    public boolean addGuidePosts( byte[] row, long byteCount, long rowCount) {
+        if (row.length != 0 && Bytes.compareTo(lastRow, row) < 0) {
+            try {
+                encoder.encode(output, row, 0, row.length);
+                this.byteCount += byteCount;
+                this.guidePostsCount++;
+                this.maxLength = encoder.getMaxLength();
+                this.rowCount += rowCount;
+                lastRow = row;
+                return true;
+            } catch (IOException e) {
+                return false;
+            }
+        }
+        return false;
+    }
+    
+    public boolean addGuidePosts(byte[] row){
+        return addGuidePosts(row, 0, 0);
+    }
+
+    public boolean addGuidePosts(byte[] row, long byteCount){
+        return addGuidePosts(row, byteCount, 0);
+    }
+
+    private void close() {
+        PrefixByteCodec.close(stream);
+    }
+
+    public GuidePostsInfo build() {
+        this.guidePosts.set(stream.getBuffer(), 0, stream.size());
+        GuidePostsInfo guidePostsInfo = new GuidePostsInfo(this.byteCount, this.guidePosts, this.rowCount, this.maxLength, this.guidePostsCount);
+        this.close();
+        return guidePostsInfo;
+    }
+    public void incrementRowCount() {
+      this.rowCount++;
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6a27cecc/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/PTableStatsImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/PTableStatsImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/PTableStatsImpl.java
index dc70e86..dacc213 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/PTableStatsImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/PTableStatsImpl.java
@@ -17,13 +17,19 @@
  */
 package org.apache.phoenix.schema.stats;
 
-import java.util.List;
+import java.io.ByteArrayInputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.EOFException;
 import java.util.Map;
 import java.util.SortedMap;
 import java.util.TreeMap;
 
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.coprocessor.MetaDataProtocol;
+import org.apache.phoenix.util.PrefixByteCodec;
+import org.apache.phoenix.util.PrefixByteDecoder;
 import org.apache.phoenix.util.SizedUtil;
 
 import com.sun.istack.NotNull;
@@ -47,12 +53,10 @@ public class PTableStatsImpl implements PTableStats {
         for (Map.Entry<byte[], GuidePostsInfo> entry : guidePosts.entrySet()) {
             byte[] cf = entry.getKey();
             estimatedSize += SizedUtil.ARRAY_SIZE + cf.length;
-            List<byte[]> keys = entry.getValue().getGuidePosts();
-            estimatedSize += SizedUtil.sizeOfArrayList(keys.size());
-            for (byte[] key : keys) {
-                estimatedSize += SizedUtil.ARRAY_SIZE + key.length;
-            }
+            estimatedSize += entry.getValue().getGuidePosts().getLength();
             estimatedSize += SizedUtil.LONG_SIZE;
+            estimatedSize += SizedUtil.INT_SIZE;
+            estimatedSize += SizedUtil.INT_SIZE;
         }
         this.estimatedSize = estimatedSize;
     }
@@ -65,19 +69,34 @@ public class PTableStatsImpl implements PTableStats {
     @Override
     public String toString() {
         StringBuilder buf = new StringBuilder();
+
         buf.append("PTableStats [");
         for (Map.Entry<byte[], GuidePostsInfo> entry : guidePosts.entrySet()) {
             buf.append(Bytes.toStringBinary(entry.getKey()));
             buf.append(":(");
-            List<byte[]> keys = entry.getValue().getGuidePosts();
-            if (!keys.isEmpty()) {
-                for (byte[] key : keys) {
-                    buf.append(Bytes.toStringBinary(key));
-                    buf.append(",");
+            ImmutableBytesWritable keys = entry.getValue().getGuidePosts();
+            ByteArrayInputStream stream = new ByteArrayInputStream(keys.get(), keys.getOffset(), keys.getLength());
+            try {
+                if (keys.getLength() != 0) {
+                    DataInput input = new DataInputStream(stream);
+                    PrefixByteDecoder decoder = new PrefixByteDecoder(entry.getValue().getMaxLength());
+                    try {
+                        while (true) {
+                            ImmutableBytesWritable ptr = PrefixByteCodec.decode(decoder, input);
+                            buf.append(Bytes.toStringBinary(ptr.get()));
+                            buf.append(",");
+                        }
+                    } catch (EOFException e) { // Ignore as this signifies we're done
+
+                    } finally {
+                        PrefixByteCodec.close(stream);
+                    }
+                    buf.setLength(buf.length() - 1);
                 }
-                buf.setLength(buf.length()-1);
+                buf.append(")");
+            } finally {
+                PrefixByteCodec.close(stream);
             }
-            buf.append(")");
         }
         buf.append("]");
         return buf.toString();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6a27cecc/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsCollector.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsCollector.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsCollector.java
index fc8d8bd..3462f22 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsCollector.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsCollector.java
@@ -19,7 +19,6 @@ package org.apache.phoenix.schema.stats;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
@@ -61,9 +60,9 @@ public class StatisticsCollector {
 
     private long guidepostDepth;
     private long maxTimeStamp = MetaDataProtocol.MIN_TABLE_TIMESTAMP;
-    private Map<ImmutableBytesPtr, Pair<Long, GuidePostsInfo>> guidePostsMap = Maps.newHashMap();
+    private Map<ImmutableBytesPtr, Pair<Long, GuidePostsInfoBuilder>> guidePostsInfoWriterMap = Maps.newHashMap();
     protected StatisticsWriter statsTable;
-    private Pair<Long, GuidePostsInfo> cachedGps = null;
+    private Pair<Long, GuidePostsInfoBuilder> cachedGps = null;
 
     public StatisticsCollector(RegionCoprocessorEnvironment env, String tableName, long clientTimeStamp)
             throws IOException {
@@ -99,8 +98,8 @@ public class StatisticsCollector {
         // in a compaction we know the one family ahead of time
         if (family != null) {
             ImmutableBytesPtr cfKey = new ImmutableBytesPtr(family);
-            cachedGps = new Pair<Long, GuidePostsInfo>(0l, new GuidePostsInfo(0, Collections.<byte[]> emptyList(), 0l));
-            guidePostsMap.put(cfKey, cachedGps);
+            cachedGps = new Pair<Long, GuidePostsInfoBuilder>(0l, new GuidePostsInfoBuilder());
+            guidePostsInfoWriterMap.put(cfKey, cachedGps);
         }
     }
 
@@ -131,7 +130,7 @@ public class StatisticsCollector {
             throws IOException {
         try {
             // update the statistics table
-            for (ImmutableBytesPtr fam : guidePostsMap.keySet()) {
+            for (ImmutableBytesPtr fam : guidePostsInfoWriterMap.keySet()) {
                 if (delete) {
                     if (logger.isDebugEnabled()) {
                         logger.debug("Deleting the stats for the region " + region.getRegionInfo());
@@ -161,22 +160,22 @@ public class StatisticsCollector {
      */
     public void collectStatistics(final List<Cell> results) {
         Map<ImmutableBytesPtr, Boolean> famMap = Maps.newHashMap();
-        List<GuidePostsInfo> rowTracker = null;
+        List<GuidePostsInfoBuilder> rowTracker = null;
         if (cachedGps == null) {
-            rowTracker = new ArrayList<GuidePostsInfo>();
+            rowTracker = new ArrayList<GuidePostsInfoBuilder>();
         }
         for (Cell cell : results) {
             KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
             maxTimeStamp = Math.max(maxTimeStamp, kv.getTimestamp());
-            Pair<Long, GuidePostsInfo> gps;
+            Pair<Long, GuidePostsInfoBuilder> gps;
             if (cachedGps == null) {
                 ImmutableBytesPtr cfKey = new ImmutableBytesPtr(kv.getFamilyArray(), kv.getFamilyOffset(),
                         kv.getFamilyLength());
-                gps = guidePostsMap.get(cfKey);
+                gps = guidePostsInfoWriterMap.get(cfKey);
                 if (gps == null) {
-                    gps = new Pair<Long, GuidePostsInfo>(0l,
-                            new GuidePostsInfo(0, Collections.<byte[]> emptyList(), 0l));
-                    guidePostsMap.put(cfKey, gps);
+                    gps = new Pair<Long, GuidePostsInfoBuilder>(0l,
+                            new GuidePostsInfoBuilder());
+                    guidePostsInfoWriterMap.put(cfKey, gps);
                 }
                 if (famMap.get(cfKey) == null) {
                     famMap.put(cfKey, true);
@@ -191,13 +190,13 @@ public class StatisticsCollector {
             if (byteCount >= guidepostDepth) {
                 byte[] row = ByteUtil.copyKeyBytesIfNecessary(
                         new ImmutableBytesWritable(kv.getRowArray(), kv.getRowOffset(), kv.getRowLength()));
-                if (gps.getSecond().addGuidePost(row, byteCount)) {
+                if (gps.getSecond().addGuidePosts(row, byteCount)) {
                     gps.setFirst(0l);
                 }
             }
         }
         if (cachedGps == null) {
-            for (GuidePostsInfo s : rowTracker) {
+            for (GuidePostsInfoBuilder s : rowTracker) {
                 s.incrementRowCount();
             }
         } else {
@@ -221,24 +220,13 @@ public class StatisticsCollector {
     }
 
     public void clear() {
-        this.guidePostsMap.clear();
+        this.guidePostsInfoWriterMap.clear();
         maxTimeStamp = MetaDataProtocol.MIN_TABLE_TIMESTAMP;
     }
 
-    public void addGuidePost(ImmutableBytesPtr cfKey, GuidePostsInfo info, long byteSize, long timestamp,
-            byte[] minKey) {
-        Pair<Long, GuidePostsInfo> newInfo = new Pair<Long, GuidePostsInfo>(byteSize, info);
-        Pair<Long, GuidePostsInfo> oldInfo = guidePostsMap.put(cfKey, newInfo);
-        if (oldInfo != null) {
-            info.combine(oldInfo.getSecond());
-            newInfo.setFirst(oldInfo.getFirst() + newInfo.getFirst());
-        }
-        maxTimeStamp = Math.max(maxTimeStamp, timestamp);
-    }
-
     public GuidePostsInfo getGuidePosts(ImmutableBytesPtr fam) {
-        Pair<Long, GuidePostsInfo> pair = guidePostsMap.get(fam);
-        if (pair != null) { return pair.getSecond(); }
+        Pair<Long, GuidePostsInfoBuilder> pair = guidePostsInfoWriterMap.get(fam);
+        if (pair != null) { return pair.getSecond().build(); }
         return null;
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6a27cecc/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsScanner.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsScanner.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsScanner.java
index d6f0bf1..13e9491 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsScanner.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsScanner.java
@@ -134,7 +134,8 @@ public class StatisticsScanner implements InternalScanner {
             } finally {
                 try {
                     collectionTracker.removeCompactingRegion(regionInfo);
-                    stats.close();
+                    stats.close();// close the writer
+                    tracker.close();// close the tracker
                 } catch (IOException e) {
                     if (toThrow == null) toThrow = e;
                     LOG.error("Error while closing the stats table", e);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6a27cecc/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsUtil.java
index 16f249c..5b47104 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsUtil.java
@@ -21,8 +21,8 @@ import static org.apache.phoenix.util.SchemaUtil.getVarCharLength;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collections;
 import java.util.List;
+import java.util.SortedMap;
 import java.util.TreeMap;
 
 import org.apache.hadoop.hbase.Cell;
@@ -55,10 +55,14 @@ public class StatisticsUtil {
 
     /** Number of parts in our complex key */
     protected static final int NUM_KEY_PARTS = 3;
-
+    
     public static byte[] getRowKey(byte[] table, ImmutableBytesPtr fam, byte[] guidePostStartKey) {
+        return getRowKey(table, fam, new ImmutableBytesWritable(guidePostStartKey,0,guidePostStartKey.length));
+    }
+
+    public static byte[] getRowKey(byte[] table, ImmutableBytesPtr fam, ImmutableBytesWritable guidePostStartKey) {
         // always starts with the source table
-        byte[] rowKey = new byte[table.length + fam.getLength() + guidePostStartKey.length + 2];
+        byte[] rowKey = new byte[table.length + fam.getLength() + guidePostStartKey.getLength() + 2];
         int offset = 0;
         System.arraycopy(table, 0, rowKey, offset, table.length);
         offset += table.length;
@@ -66,7 +70,7 @@ public class StatisticsUtil {
         System.arraycopy(fam.get(), fam.getOffset(), rowKey, offset, fam.getLength());
         offset += fam.getLength();
         rowKey[offset++] = QueryConstants.SEPARATOR_BYTE; // assumes stats table columns not DESC
-        System.arraycopy(guidePostStartKey, 0, rowKey, offset, guidePostStartKey.length);
+        System.arraycopy(guidePostStartKey.get(), 0, rowKey, offset, guidePostStartKey.getLength());
         return rowKey;
     }
 
@@ -126,7 +130,7 @@ public class StatisticsUtil {
         s.addColumn(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, QueryConstants.EMPTY_COLUMN_BYTES);
         ResultScanner scanner = null;
         long timeStamp = MetaDataProtocol.MIN_TABLE_TIMESTAMP;
-        TreeMap<byte[], GuidePostsInfo> guidePostsPerCf = new TreeMap<byte[], GuidePostsInfo>(Bytes.BYTES_COMPARATOR);
+        TreeMap<byte[], GuidePostsInfoBuilder> guidePostsInfoWriterPerCf = new TreeMap<byte[], GuidePostsInfoBuilder>(Bytes.BYTES_COMPARATOR);
         try {
             scanner = statsHTable.getScanner(s);
             Result result = null;
@@ -168,23 +172,33 @@ public class StatisticsUtil {
                 }
                 if (cfName != null) {
                     byte[] newGPStartKey = getGuidePostsInfoFromRowKey(tableNameBytes, cfName, result.getRow());
-                    GuidePostsInfo guidePosts = guidePostsPerCf.get(cfName);
-                    if (guidePosts == null) {
-                        guidePosts = new GuidePostsInfo(0l, Collections.<byte[]> emptyList(), 0l);
-                        guidePostsPerCf.put(cfName, guidePosts);
+                    GuidePostsInfoBuilder guidePostsInfoWriter = guidePostsInfoWriterPerCf.get(cfName);
+                    if (guidePostsInfoWriter == null) {
+                        guidePostsInfoWriter = new GuidePostsInfoBuilder();
+                        guidePostsInfoWriterPerCf.put(cfName, guidePostsInfoWriter);
                     }
-                    guidePosts.addGuidePost(newGPStartKey, byteCount, rowCount);
+                    guidePostsInfoWriter.addGuidePosts(newGPStartKey, byteCount, rowCount);
                 }
             }
+            if (!guidePostsInfoWriterPerCf.isEmpty()) { return new PTableStatsImpl(
+                    getGuidePostsPerCf(guidePostsInfoWriterPerCf), timeStamp); }
         } finally {
             if (scanner != null) {
                 scanner.close();
             }
         }
-        if (!guidePostsPerCf.isEmpty()) { return new PTableStatsImpl(guidePostsPerCf, timeStamp); }
         return PTableStats.EMPTY_STATS;
     }
 
+    private static SortedMap<byte[], GuidePostsInfo> getGuidePostsPerCf(
+            TreeMap<byte[], GuidePostsInfoBuilder> guidePostsWriterPerCf) {
+        TreeMap<byte[], GuidePostsInfo> guidePostsPerCf = new TreeMap<byte[], GuidePostsInfo>(Bytes.BYTES_COMPARATOR);
+        for (byte[] key : guidePostsWriterPerCf.keySet()) {
+            guidePostsPerCf.put(key, guidePostsWriterPerCf.get(key).build());
+        }
+        return guidePostsPerCf;
+    }
+
     public static long getGuidePostDepth(int guidepostPerRegion, long guidepostWidth, HTableDescriptor tableDesc) {
         if (guidepostPerRegion > 0) {
             long maxFileSize = HConstants.DEFAULT_MAX_FILE_SIZE;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6a27cecc/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsWriter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsWriter.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsWriter.java
index b8bd064..d03af7a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsWriter.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsWriter.java
@@ -17,7 +17,11 @@
  */
 package org.apache.phoenix.schema.stats;
 
+import java.io.ByteArrayInputStream;
 import java.io.Closeable;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.EOFException;
 import java.io.IOException;
 import java.sql.Date;
 import java.util.List;
@@ -45,6 +49,8 @@ import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.types.PDate;
 import org.apache.phoenix.schema.types.PLong;
 import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.PrefixByteCodec;
+import org.apache.phoenix.util.PrefixByteDecoder;
 import org.apache.phoenix.util.ServerUtil;
 import org.apache.phoenix.util.TimeKeeper;
 
@@ -103,6 +109,7 @@ public class StatisticsWriter implements Closeable {
     @Override
     public void close() throws IOException {
         statsWriterTable.close();
+        statsReaderTable.close();
     }
 
     /**
@@ -134,21 +141,32 @@ public class StatisticsWriter implements Closeable {
         GuidePostsInfo gps = tracker.getGuidePosts(cfKey);
         if (gps != null) {
             boolean rowColumnAdded = false;
-            for (byte[] gp : gps.getGuidePosts()) {
-                byte[] prefix = StatisticsUtil.getRowKey(tableName, cfKey, gp);
-                Put put = new Put(prefix);
-                if (!rowColumnAdded) {
-                    put.add(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH_BYTES,
-                            timeStamp, PLong.INSTANCE.toBytes(gps.getByteCount()));
-                    put.add(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES,
-                            PhoenixDatabaseMetaData.GUIDE_POSTS_ROW_COUNT_BYTES, timeStamp,
-                            PLong.INSTANCE.toBytes(gps.getRowCount()));
-                    rowColumnAdded = true;
+            ImmutableBytesWritable keys = gps.getGuidePosts();
+            ByteArrayInputStream stream = new ByteArrayInputStream(keys.get(), keys.getOffset(), keys.getLength());
+            DataInput input = new DataInputStream(stream);
+            PrefixByteDecoder decoder = new PrefixByteDecoder(gps.getMaxLength());
+            try {
+                while (true) {
+                    ImmutableBytesWritable ptr = decoder.decode(input);
+                    byte[] prefix = StatisticsUtil.getRowKey(tableName, cfKey, ptr);
+                    Put put = new Put(prefix);
+                    if (!rowColumnAdded) {
+                        put.add(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH_BYTES,
+                                timeStamp, PLong.INSTANCE.toBytes(gps.getByteCount()));
+                        put.add(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES,
+                                PhoenixDatabaseMetaData.GUIDE_POSTS_ROW_COUNT_BYTES, timeStamp,
+                                PLong.INSTANCE.toBytes(gps.getRowCount()));
+                        rowColumnAdded = true;
+                    }
+                    // Add our empty column value so queries behave correctly
+                    put.add(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, QueryConstants.EMPTY_COLUMN_BYTES, timeStamp,
+                            ByteUtil.EMPTY_BYTE_ARRAY);
+                    mutations.add(put);
                 }
-                // Add our empty column value so queries behave correctly
-                put.add(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, QueryConstants.EMPTY_COLUMN_BYTES, timeStamp,
-                        ByteUtil.EMPTY_BYTE_ARRAY);
-                mutations.add(put);
+            } catch (EOFException e) { // Ignore as this signifies we're done
+
+            } finally {
+                PrefixByteCodec.close(stream);
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6a27cecc/phoenix-core/src/main/java/org/apache/phoenix/util/ByteUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/ByteUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/ByteUtil.java
index 64d064a..44502a5 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/ByteUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/ByteUtil.java
@@ -49,6 +49,8 @@ public class ByteUtil {
     public static final byte[] EMPTY_BYTE_ARRAY = new byte[0];
     public static final ImmutableBytesPtr EMPTY_BYTE_ARRAY_PTR = new ImmutableBytesPtr(
             EMPTY_BYTE_ARRAY);
+    public static final ImmutableBytesWritable EMPTY_IMMUTABLE_BYTE_ARRAY = new ImmutableBytesWritable(
+            EMPTY_BYTE_ARRAY);
     
     public static final Comparator<ImmutableBytesPtr> BYTES_PTR_COMPARATOR = new Comparator<ImmutableBytesPtr>() {
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6a27cecc/phoenix-core/src/main/java/org/apache/phoenix/util/PrefixByteCodec.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/PrefixByteCodec.java b/phoenix-core/src/main/java/org/apache/phoenix/util/PrefixByteCodec.java
new file mode 100644
index 0000000..8c3aa80
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/PrefixByteCodec.java
@@ -0,0 +1,104 @@
+/*
+ * 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.phoenix.util;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.EOFException;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+import com.google.common.collect.Lists;
+
+public class PrefixByteCodec {
+    
+    public static List<byte[]> decodeBytes(ImmutableBytesWritable encodedBytes, int maxLength) throws IOException {
+        ByteArrayInputStream stream = new ByteArrayInputStream(encodedBytes.get(), encodedBytes.getOffset(), encodedBytes.getLength());
+        DataInput input = new DataInputStream(stream);
+        PrefixByteDecoder decoder = new PrefixByteDecoder(maxLength);
+        List<byte[]> listOfBytes = Lists.newArrayList();
+        try {
+            while (true) {
+                ImmutableBytesWritable ptr = decoder.decode(input);
+                // For this test, copy the bytes, but we wouldn't do this unless
+                // necessary for non testing
+                listOfBytes.add(ptr.copyBytes());
+            }
+        } catch (EOFException e) { // Ignore as this signifies we're done
+            
+        }
+        return listOfBytes;
+    }
+    
+    public static int encodeBytes(List<byte[]> listOfBytes, ImmutableBytesWritable ptr) throws IOException {
+        TrustedByteArrayOutputStream stream = new TrustedByteArrayOutputStream(calculateSize(listOfBytes));
+        DataOutput output = new DataOutputStream(stream);
+        PrefixByteEncoder encoder = new PrefixByteEncoder();
+        for (byte[] bytes : listOfBytes) {
+            encoder.encode(output, bytes, 0, bytes.length);
+        }
+        close(stream);
+        ptr.set(stream.getBuffer(), 0, stream.size());
+        return encoder.getMaxLength();
+    }
+    
+    public static int calculateSize(List<byte[]> listOfBytes) {
+        int size = 0;
+        for (byte[] bytes : listOfBytes) {
+            size += bytes.length;
+        }
+        return size;
+    }
+
+    public static ImmutableBytesWritable decode(PrefixByteDecoder decoder, DataInput input) throws EOFException {
+        try {
+            ImmutableBytesWritable val= decoder.decode(input);
+            return val;
+        } catch(EOFException eof){
+            throw eof;
+        }catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public static void close(ByteArrayInputStream stream) {
+        if (stream != null) {
+            try {
+                stream.close();
+            } catch (IOException e) {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
+    public static void close(ByteArrayOutputStream stream) {
+        if (stream != null) {
+            try {
+                stream.close();
+            } catch (IOException e) {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6a27cecc/phoenix-core/src/main/java/org/apache/phoenix/util/PrefixByteDecoder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/PrefixByteDecoder.java b/phoenix-core/src/main/java/org/apache/phoenix/util/PrefixByteDecoder.java
new file mode 100644
index 0000000..c34bda8
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/PrefixByteDecoder.java
@@ -0,0 +1,90 @@
+/*
+ * 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.phoenix.util;
+
+import java.io.DataInput;
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.io.WritableUtils;
+
+/**
+ * 
+ * Prefix decoder for byte arrays. For encoding, see {@link PrefixByteEncoder}.
+ * 
+ */
+public class PrefixByteDecoder {
+    private final int maxLength;
+    private final ImmutableBytesWritable previous;
+    
+    /**
+     * Used when the maximum length of encoded byte array is not known. Will
+     * cause a new byte array to be allocated for each call to {@link #decode(DataInput)}.
+     */
+    public PrefixByteDecoder() {
+        maxLength = -1;
+        previous = new ImmutableBytesWritable(ByteUtil.EMPTY_BYTE_ARRAY);
+    }
+
+    /**
+     * Used when the maximum length of encoded byte array is known in advance. 
+     * Will not allocate new byte array with each call to {@link #decode(DataInput)}.
+     * @param maxLength maximum length needed for any call to {@link #decode(DataInput)}.
+     */
+    public PrefixByteDecoder(int maxLength) {
+        if (maxLength > 0) {
+            this.maxLength = maxLength;
+            this.previous = new ImmutableBytesWritable(new byte[maxLength], 0, 0);
+        } else {
+            this.maxLength = -1;
+            previous = new ImmutableBytesWritable(ByteUtil.EMPTY_BYTE_ARRAY);
+        }
+    }
+    
+    /**
+     * Resets state of decoder if it will be used to decode bytes from a
+     * different DataInput.
+     */
+    public void reset() {
+        previous.set(previous.get(),0,0);
+    }
+    
+    /**
+     * Decodes bytes encoded with {@link PrefixByteEncoder}.
+     * @param in Input from which bytes are read.
+     * @return Pointer containing bytes that were decoded. Note that the
+     * same pointer will be returned with each call, so it must be consumed
+     * prior to calling decode again.
+     * @throws IOException
+     */
+    public ImmutableBytesWritable decode(DataInput in) throws IOException {
+        int prefixLen = WritableUtils.readVInt(in);
+        int suffixLen = WritableUtils.readVInt(in);
+        int length = prefixLen + suffixLen;
+        byte[] b;
+        if (maxLength == -1) { // Allocate new byte array each time
+            b = new byte[length];
+            System.arraycopy(previous.get(), previous.getOffset(), b, 0, prefixLen);
+        } else { // Reuse same buffer each time
+            b = previous.get();
+        }
+        in.readFully(b, prefixLen, suffixLen);
+        previous.set(b, 0, length);
+        return previous;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6a27cecc/phoenix-core/src/main/java/org/apache/phoenix/util/PrefixByteEncoder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/PrefixByteEncoder.java b/phoenix-core/src/main/java/org/apache/phoenix/util/PrefixByteEncoder.java
new file mode 100644
index 0000000..bf92be5
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/PrefixByteEncoder.java
@@ -0,0 +1,99 @@
+/*
+ * 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.phoenix.util;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.WritableUtils;
+
+/**
+ * 
+ * Prefix encoder for byte arrays. For decoding, see {@link PrefixByteDecoder}.
+ *
+ */
+public class PrefixByteEncoder {
+    private int maxLength;
+    private final ImmutableBytesWritable previous = new ImmutableBytesWritable(ByteUtil.EMPTY_BYTE_ARRAY);
+    
+    public PrefixByteEncoder() {
+    }
+
+    /**
+     * Resets the state of the encoder to its initial state (i.e. forgetting
+     * the previous byte array that may have been encoded).
+     */
+    public void reset() {
+        previous.set(ByteUtil.EMPTY_BYTE_ARRAY);
+    }
+    
+    /**
+     * @return the maximum length byte array encountered while encoding
+     */
+    public int getMaxLength() {
+        return maxLength;
+    }
+    
+    /**
+     * Prefix encodes the byte array pointed to into the output stream
+     * @param out output stream to encode into
+     * @param ptr pointer to byte array to encode.
+     * @throws IOException
+     */
+    public void encode(DataOutput out, ImmutableBytesWritable ptr) throws IOException {
+        encode(out, ptr.get(), ptr.getOffset(), ptr.getLength());
+    }
+    
+    /**
+     * Prefix encodes the byte array into the output stream
+     * @param out output stream to encode into
+     * @param b byte array to encode
+     * @throws IOException
+     */
+    public void encode(DataOutput out, byte[] b) throws IOException {
+        encode(out, b, 0, b.length);
+    }
+
+    /**
+     * Prefix encodes the byte array from offset to length into output stream. 
+     * Instead of writing the entire byte array, only the portion of the byte array
+     * that differs from the beginning of the previous byte array written is written.
+     *  
+     * @param out output stream to encode into
+     * @param b byte array buffer
+     * @param offset offset into byte array to start encoding
+     * @param length length of byte array to encode
+     * @throws IOException
+     */
+    public void encode(DataOutput out, byte[] b, int offset, int length) throws IOException {
+          int i = 0;
+          int prevOffset = previous.getOffset();
+          byte[] prevBytes = previous.get();
+          int prevLength = previous.getLength();
+          int minLength = prevLength < b.length ? prevLength : b.length;
+          for(i = 0; (i < minLength) && (prevBytes[prevOffset + i] == b[offset + i]); i++);
+          WritableUtils.writeVInt(out, i);
+          Bytes.writeByteArray(out, b, offset + i, length - i);
+          previous.set(b, offset, length);
+          if (length > maxLength) {
+              maxLength = length;
+          }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6a27cecc/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
index c931851..1d6f438 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
@@ -52,12 +52,14 @@ import java.util.Set;
 import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
 
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Mutation;
@@ -100,6 +102,7 @@ import com.google.common.collect.Sets;
 public class UpgradeUtil {
     private static final Logger logger = LoggerFactory.getLogger(UpgradeUtil.class);
     private static final byte[] SEQ_PREFIX_BYTES = ByteUtil.concat(QueryConstants.SEPARATOR_BYTE_ARRAY, Bytes.toBytes("_SEQ_"));
+    public static final byte[] UPGRADE_TO_4_7_COLUMN_NAME = Bytes.toBytes("UPGRADE_TO_4_7");
     
     public static String UPSERT_BASE_COLUMN_COUNT_IN_HEADER_ROW = "UPSERT "
             + "INTO SYSTEM.CATALOG "
@@ -1215,4 +1218,67 @@ public class UpgradeUtil {
                 MetaDataEndpointImpl.ROW_KEY_ORDER_OPTIMIZABLE_BYTES, PBoolean.INSTANCE.toBytes(true));
         tableMetadata.add(put);
     }
-}
+
+    public static boolean truncateStats(HTableInterface metaTable, HTableInterface statsTable)
+            throws IOException, InterruptedException {
+        List<Cell> columnCells = metaTable
+                .get(new Get(SchemaUtil.getTableKey(null, PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME,
+                        PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE)))
+                .getColumnCells(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES, QueryConstants.EMPTY_COLUMN_BYTES);
+        if (!columnCells.isEmpty()
+                && columnCells.get(0).getTimestamp() < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0) {
+
+            byte[] statsTableKey = SchemaUtil.getTableKey(null, PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME,
+                    PhoenixDatabaseMetaData.SYSTEM_STATS_TABLE);
+            KeyValue upgradeKV = KeyValueUtil.newKeyValue(statsTableKey, PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
+                    UPGRADE_TO_4_7_COLUMN_NAME, MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 1,
+                    ByteUtil.EMPTY_BYTE_ARRAY);
+            Put upgradePut = new Put(statsTableKey);
+            upgradePut.add(upgradeKV);
+
+            // check for null in UPGRADE_TO_4_7_COLUMN_NAME in checkAndPut so that only single client
+            // drop the rows of SYSTEM.STATS
+            if (metaTable.checkAndPut(statsTableKey, PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
+                    UPGRADE_TO_4_7_COLUMN_NAME, null, upgradePut)) {
+                List<Mutation> mutations = Lists.newArrayListWithExpectedSize(1000);
+                Scan scan = new Scan();
+                scan.setRaw(true);
+                scan.setMaxVersions();
+                ResultScanner statsScanner = statsTable.getScanner(scan);
+                Result r;
+                mutations.clear();
+                int count = 0;
+                while ((r = statsScanner.next()) != null) {
+                    Delete delete = null;
+                    for (KeyValue keyValue : r.raw()) {
+                        if (KeyValue.Type.codeToType(keyValue.getType()) == KeyValue.Type.Put) {
+                            if (delete == null) {
+                                delete = new Delete(keyValue.getRow());
+                            }
+                            KeyValue deleteKeyValue = new KeyValue(keyValue.getRowArray(), keyValue.getRowOffset(),
+                                    keyValue.getRowLength(), keyValue.getFamilyArray(), keyValue.getFamilyOffset(),
+                                    keyValue.getFamilyLength(), keyValue.getQualifierArray(),
+                                    keyValue.getQualifierOffset(), keyValue.getQualifierLength(),
+                                    keyValue.getTimestamp(), KeyValue.Type.Delete, ByteUtil.EMPTY_BYTE_ARRAY, 0, 0);
+                            delete.addDeleteMarker(deleteKeyValue);
+                        }
+                    }
+                    if (delete != null) {
+                        mutations.add(delete);
+                        if (count > 10) {
+                            statsTable.batch(mutations);
+                            mutations.clear();
+                            count = 0;
+                        }
+                        count++;
+                    }
+                }
+                if (!mutations.isEmpty()) {
+                    statsTable.batch(mutations);
+                }
+                return true;
+            }
+        }
+        return false;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6a27cecc/phoenix-core/src/test/java/org/apache/phoenix/filter/SkipScanBigFilterTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/filter/SkipScanBigFilterTest.java b/phoenix-core/src/test/java/org/apache/phoenix/filter/SkipScanBigFilterTest.java
index a0aca4d..bb5f408 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/filter/SkipScanBigFilterTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/filter/SkipScanBigFilterTest.java
@@ -22,12 +22,10 @@ import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
 import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
-import java.util.Collections;
 import java.util.Map;
 import java.util.Properties;
 import java.util.SortedMap;
 
-import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.compile.QueryPlan;
 import org.apache.phoenix.end2end.Shadower;
@@ -40,6 +38,7 @@ import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableImpl;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.stats.GuidePostsInfo;
+import org.apache.phoenix.schema.stats.GuidePostsInfoBuilder;
 import org.apache.phoenix.schema.stats.PTableStats;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
@@ -648,10 +647,11 @@ public class SkipScanBigFilterTest extends BaseConnectionlessQueryTest {
         stmt.execute();
         
         final PTable table = conn.unwrap(PhoenixConnection.class).getTable(new PTableKey(null, "PERF.BIG_OLAP_DOC"));
-        GuidePostsInfo info = new GuidePostsInfo(0,Collections.<byte[]> emptyList(), 0l);
+        GuidePostsInfoBuilder gpWriter = new GuidePostsInfoBuilder();
         for (byte[] gp : guidePosts) {
-            info.addGuidePost(gp, 1000);
+            gpWriter.addGuidePosts(gp, 1000);
         }
+        GuidePostsInfo info = gpWriter.build();
         final SortedMap<byte[], GuidePostsInfo> gpMap = Maps.newTreeMap(Bytes.BYTES_COMPARATOR);
         gpMap.put(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, info);
         PTable tableWithStats = PTableImpl.makePTable(table, new PTableStats() {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6a27cecc/phoenix-core/src/test/java/org/apache/phoenix/util/PrefixByteEncoderDecoderTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/PrefixByteEncoderDecoderTest.java b/phoenix-core/src/test/java/org/apache/phoenix/util/PrefixByteEncoderDecoderTest.java
new file mode 100644
index 0000000..f8aa7db
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/util/PrefixByteEncoderDecoderTest.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.util;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.phoenix.query.QueryConstants;
+import org.junit.Test;
+
+
+public class PrefixByteEncoderDecoderTest {
+
+    static final List<byte[]> guideposts = Arrays.asList(
+            ByteUtil.concat(Bytes.toBytes("aaaaaaaaaa"), QueryConstants.SEPARATOR_BYTE_ARRAY, Bytes.toBytes(1000L), Bytes.toBytes("bbbbbbbbbb")),
+            ByteUtil.concat(Bytes.toBytes("aaaaaaaaaa"), QueryConstants.SEPARATOR_BYTE_ARRAY, Bytes.toBytes(1000L), Bytes.toBytes("bbbbbccccc")),
+            ByteUtil.concat(Bytes.toBytes("aaaaaaaaaa"), QueryConstants.SEPARATOR_BYTE_ARRAY, Bytes.toBytes(2000L), Bytes.toBytes("bbbbbbbbbb")),
+            ByteUtil.concat(Bytes.toBytes("bbbbbbbbbb"), QueryConstants.SEPARATOR_BYTE_ARRAY, Bytes.toBytes(1000L), Bytes.toBytes("bbbbbbbbbb")),
+            ByteUtil.concat(Bytes.toBytes("bbbbbbbbbb"), QueryConstants.SEPARATOR_BYTE_ARRAY, Bytes.toBytes(2000L), Bytes.toBytes("bbbbbbbbbb")),
+            ByteUtil.concat(Bytes.toBytes("bbbbbbbbbb"), QueryConstants.SEPARATOR_BYTE_ARRAY, Bytes.toBytes(2000L), Bytes.toBytes("c")),
+            ByteUtil.concat(Bytes.toBytes("bbbbbbbbbbb"), QueryConstants.SEPARATOR_BYTE_ARRAY, Bytes.toBytes(1000L), Bytes.toBytes("bbbbbbbbbb")),
+            ByteUtil.concat(Bytes.toBytes("d"), QueryConstants.SEPARATOR_BYTE_ARRAY, Bytes.toBytes(1000L), Bytes.toBytes("bbbbbbbbbb")),
+            ByteUtil.concat(Bytes.toBytes("d"), QueryConstants.SEPARATOR_BYTE_ARRAY, Bytes.toBytes(1000L), Bytes.toBytes("bbbbbbbbbbc")),
+            ByteUtil.concat(Bytes.toBytes("e"), QueryConstants.SEPARATOR_BYTE_ARRAY, Bytes.toBytes(1000L), Bytes.toBytes("bbbbbbbbbb"))
+            );
+    
+    @Test
+    public void testEncode() throws IOException {
+        List<byte[]> listOfBytes = Arrays.asList(Bytes.toBytes("aaaaa"), Bytes.toBytes("aaaabb"));
+        ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+        int maxLength = PrefixByteCodec.encodeBytes(listOfBytes, ptr);
+        assertEquals(6, maxLength);
+        TrustedByteArrayOutputStream stream = new TrustedByteArrayOutputStream(PrefixByteCodec.calculateSize(listOfBytes));
+        DataOutput output = new DataOutputStream(stream);
+        WritableUtils.writeVInt(output, 0);
+        WritableUtils.writeVInt(output, 5);
+        output.write(Bytes.toBytes("aaaaa")); // No space savings on first key
+        WritableUtils.writeVInt(output, 4);
+        WritableUtils.writeVInt(output, 2);
+        output.write(Bytes.toBytes("bb")); // Only writes part of second key that's different
+        assertArrayEquals(stream.toByteArray(), ptr.copyBytes());
+    }
+    
+    @Test
+    public void testEncodeDecodeWithSingleBuffer() throws IOException {
+        testEncodeDecode(true);
+    }
+    
+    @Test
+    public void testEncodeDecodeWithNewBuffer() throws IOException {
+        testEncodeDecode(false);
+    }
+    
+    private void testEncodeDecode(boolean useSingleBuffer) throws IOException {
+        ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+        int maxLength = PrefixByteCodec.encodeBytes(guideposts, ptr);
+        int encodedSize = ptr.getLength();
+        int unencodedSize = PrefixByteCodec.calculateSize(guideposts);
+        assertTrue(encodedSize < unencodedSize);
+        List<byte[]> listOfBytes = PrefixByteCodec.decodeBytes(ptr, useSingleBuffer ? maxLength : -1);
+        assertListByteArraysEquals(guideposts, listOfBytes);
+    }
+    
+    private static void assertListByteArraysEquals(List<byte[]> listOfBytes1, List<byte[]> listOfBytes2) {
+        assertEquals(listOfBytes1.size(), listOfBytes2.size());
+        for (int i = 0; i < listOfBytes1.size(); i++) {
+            assertArrayEquals(listOfBytes1.get(i), listOfBytes2.get(i));
+        }
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6a27cecc/phoenix-protocol/src/main/PGuidePosts.proto
----------------------------------------------------------------------
diff --git a/phoenix-protocol/src/main/PGuidePosts.proto b/phoenix-protocol/src/main/PGuidePosts.proto
index 047a658..14de2eb 100644
--- a/phoenix-protocol/src/main/PGuidePosts.proto
+++ b/phoenix-protocol/src/main/PGuidePosts.proto
@@ -27,4 +27,7 @@ message PGuidePosts {
   repeated bytes guidePosts = 1;
   optional int64 byteCount = 2;
   optional int64 rowCount = 3;
+  optional int32 maxLength = 4;
+  optional int32 encodedGuidePostsCount = 5;
+  optional bytes encodedGuidePosts = 6;
 }
\ No newline at end of file


[25/50] [abbrv] phoenix git commit: PHOENIX-2446 Immutable index - Index vs base table row count does not match when index is created during data load

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/a138cfe0/phoenix-core/src/test/java/org/apache/phoenix/expression/ColumnExpressionTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/expression/ColumnExpressionTest.java b/phoenix-core/src/test/java/org/apache/phoenix/expression/ColumnExpressionTest.java
index 7a299a9..7ee579c 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/expression/ColumnExpressionTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/expression/ColumnExpressionTest.java
@@ -41,7 +41,7 @@ public class ColumnExpressionTest {
         int maxLen = 30;
         int scale = 5;
         PColumn column = new PColumnImpl(PNameFactory.newName("c1"), PNameFactory.newName("f1"), PDecimal.INSTANCE, maxLen, scale,
-                true, 20, SortOrder.getDefault(), 0, null, false, null, false);
+                true, 20, SortOrder.getDefault(), 0, null, false, null, false, false);
         ColumnExpression colExp = new KeyValueColumnExpression(column);
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
         DataOutputStream dOut = new DataOutputStream(baos);
@@ -61,7 +61,7 @@ public class ColumnExpressionTest {
     public void testSerializationWithNullScale() throws Exception {
         int maxLen = 30;
         PColumn column = new PColumnImpl(PNameFactory.newName("c1"), PNameFactory.newName("f1"), PBinary.INSTANCE, maxLen, null,
-                true, 20, SortOrder.getDefault(), 0, null, false, null, false);
+                true, 20, SortOrder.getDefault(), 0, null, false, null, false, false);
         ColumnExpression colExp = new KeyValueColumnExpression(column);
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
         DataOutputStream dOut = new DataOutputStream(baos);
@@ -81,7 +81,7 @@ public class ColumnExpressionTest {
     public void testSerializationWithNullMaxLength() throws Exception {
         int scale = 5;
         PColumn column = new PColumnImpl(PNameFactory.newName("c1"), PNameFactory.newName("f1"), PVarchar.INSTANCE, null, scale,
-                true, 20, SortOrder.getDefault(), 0, null, false, null, false);
+                true, 20, SortOrder.getDefault(), 0, null, false, null, false, false);
         ColumnExpression colExp = new KeyValueColumnExpression(column);
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
         DataOutputStream dOut = new DataOutputStream(baos);
@@ -100,7 +100,7 @@ public class ColumnExpressionTest {
     @Test
     public void testSerializationWithNullScaleAndMaxLength() throws Exception {
         PColumn column = new PColumnImpl(PNameFactory.newName("c1"), PNameFactory.newName("f1"), PDecimal.INSTANCE, null, null, true,
-                20, SortOrder.getDefault(), 0, null, false, null, false);
+                20, SortOrder.getDefault(), 0, null, false, null, false, false);
         ColumnExpression colExp = new KeyValueColumnExpression(column);
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
         DataOutputStream dOut = new DataOutputStream(baos);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a138cfe0/phoenix-core/src/test/java/org/apache/phoenix/iterate/AggregateResultScannerTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/iterate/AggregateResultScannerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/iterate/AggregateResultScannerTest.java
index b6371f1..f53e871 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/iterate/AggregateResultScannerTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/iterate/AggregateResultScannerTest.java
@@ -133,6 +133,10 @@ public class AggregateResultScannerTest extends BaseConnectionlessQueryTest {
             public boolean isRowTimestamp() {
                 return false;
             }
+			@Override
+			public boolean isDynamic() {
+				return false;
+			}
         })), null);
         aggregationManager.setAggregators(new ClientAggregators(Collections.<SingleAggregateFunction>singletonList(func), 1));
         ResultIterators iterators = new ResultIterators() {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a138cfe0/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java b/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java
index 59c8b02..29a7001 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java
@@ -62,6 +62,7 @@ public final class QueryServicesTestImpl extends BaseQueryServicesImpl {
     public static final int DEFAULT_HCONNECTION_POOL_CORE_SIZE = 10;
     public static final int DEFAULT_HCONNECTION_POOL_MAX_SIZE = 10;
     public static final int DEFAULT_HTABLE_MAX_THREADS = 10;
+    public static final long DEFAULT_INDEX_POPULATION_WAIT_TIME = 0;
 
     
     /**
@@ -107,7 +108,8 @@ public final class QueryServicesTestImpl extends BaseQueryServicesImpl {
                 .setMetadataHandlerCount(DEFAULT_METADATA_HANDLER_COUNT)
                 .setHConnectionPoolCoreSize(DEFAULT_HCONNECTION_POOL_CORE_SIZE)
                 .setHConnectionPoolMaxSize(DEFAULT_HCONNECTION_POOL_MAX_SIZE)
-                .setMaxThreadsPerHTable(DEFAULT_HTABLE_MAX_THREADS);
+                .setMaxThreadsPerHTable(DEFAULT_HTABLE_MAX_THREADS)
+                .setDefaultIndexPopulationWaitTime(DEFAULT_INDEX_POPULATION_WAIT_TIME);
     }
     
     public QueryServicesTestImpl(ReadOnlyProps defaultProps, ReadOnlyProps overrideProps) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a138cfe0/phoenix-protocol/src/main/MetaDataService.proto
----------------------------------------------------------------------
diff --git a/phoenix-protocol/src/main/MetaDataService.proto b/phoenix-protocol/src/main/MetaDataService.proto
index c631512..a5a3934 100644
--- a/phoenix-protocol/src/main/MetaDataService.proto
+++ b/phoenix-protocol/src/main/MetaDataService.proto
@@ -61,7 +61,7 @@ message GetTableRequest {
   required bytes tableName = 3;
   required int64 tableTimestamp = 4;
   required int64 clientTimestamp = 5;
-  optional int32 clientVersion = 6;
+  optional int32 clientVersion = 6;	
 }
 
 message GetFunctionsRequest {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a138cfe0/phoenix-protocol/src/main/PTable.proto
----------------------------------------------------------------------
diff --git a/phoenix-protocol/src/main/PTable.proto b/phoenix-protocol/src/main/PTable.proto
index 485dd11..8bdb57f 100644
--- a/phoenix-protocol/src/main/PTable.proto
+++ b/phoenix-protocol/src/main/PTable.proto
@@ -48,6 +48,7 @@ message PColumn {
   optional bool viewReferenced = 11;
   optional string expression = 12;
   optional bool isRowTimestamp = 13;
+  optional bool isDynamic = 14;
 }
 
 message PTableStats {


[24/50] [abbrv] phoenix git commit: PHOENIX-2584 - Support Array datatype in phoenix-pig module

Posted by ma...@apache.org.
PHOENIX-2584 - Support Array datatype in phoenix-pig module


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

Branch: refs/heads/calcite
Commit: 8574d4311dcb5fc7a76465b32f24daf045ffb562
Parents: 4625096
Author: ravimagham <ra...@bazaarvoice.com>
Authored: Thu Jan 21 18:31:28 2016 -0800
Committer: ravimagham <ra...@bazaarvoice.com>
Committed: Thu Jan 21 18:31:28 2016 -0800

----------------------------------------------------------------------
 .../mapreduce/PhoenixRecordWritable.java        |  28 ++--
 .../phoenix/pig/PhoenixHBaseLoaderIT.java       | 167 ++++++++++++++++++-
 .../phoenix/pig/PhoenixHBaseStorerIT.java       |  37 ++++
 .../apache/phoenix/pig/PhoenixHBaseLoader.java  |  22 +--
 .../apache/phoenix/pig/PhoenixHBaseStorage.java |  33 +++-
 .../org/apache/phoenix/pig/util/TypeUtil.java   |  49 ++++--
 .../pig/writable/PhoenixPigDBWritable.java      | 121 --------------
 .../apache/phoenix/pig/util/TypeUtilTest.java   |  39 +++--
 8 files changed, 319 insertions(+), 177 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/8574d431/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/PhoenixRecordWritable.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/PhoenixRecordWritable.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/PhoenixRecordWritable.java
index 8d7d97a..0d3e724 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/PhoenixRecordWritable.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/PhoenixRecordWritable.java
@@ -17,11 +17,6 @@
  */
 package org.apache.phoenix.mapreduce;
 
-import org.apache.hadoop.mapreduce.lib.db.DBWritable;
-import org.apache.phoenix.schema.types.*;
-import org.apache.phoenix.util.ColumnInfo;
-import org.joda.time.DateTime;
-
 import java.sql.Array;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
@@ -29,16 +24,24 @@ import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.HashMap;
+import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.hadoop.mapreduce.lib.db.DBWritable;
+import org.apache.phoenix.schema.types.PBinary;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PDate;
+import org.apache.phoenix.schema.types.PVarbinary;
+import org.apache.phoenix.util.ColumnInfo;
+import org.joda.time.DateTime;
+
 
 public class PhoenixRecordWritable implements DBWritable {
 
     private final List<Object> upsertValues = new ArrayList<>();
-    private final Map<String, Object> resultMap = new HashMap<>();
-    private List<ColumnInfo> columnMetaDataList;
+    private final Map<String, Object> resultMap = new LinkedHashMap<>();
+    private List<ColumnInfo> columnMetaDataList; 
 
     /** For serialization; do not use. */
     public PhoenixRecordWritable() {
@@ -147,9 +150,10 @@ public class PhoenixRecordWritable implements DBWritable {
                 // PVarbinary and PBinary are provided as byte[] but are treated as SQL objects
                 if (PDataType.equalsAny(finalType, PVarbinary.INSTANCE, PBinary.INSTANCE)) {
                     statement.setObject(i + 1, finalObj);
+                } else {
+                    // otherwise set as array type
+                    setArrayInStatement(statement, finalType, primativeArrayToObjectArray((byte[]) finalObj), i + 1);
                 }
-                // otherwise set as array type
-                setArrayInStatement(statement, finalType, primativeArrayToObjectArray((byte[]) finalObj), i + 1);
             } else if (finalObj instanceof short[]) {
                 setArrayInStatement(statement, finalType, primativeArrayToObjectArray((short[]) finalObj), i + 1);
             } else if (finalObj instanceof int[]) {
@@ -171,10 +175,6 @@ public class PhoenixRecordWritable implements DBWritable {
         for (int i = 1; i <= metaData.getColumnCount(); i++) {
             // return the contents of a PhoenixArray, if necessary
             Object value = resultSet.getObject(i);
-            if (value instanceof PhoenixArray) {
-                value = ((PhoenixArray) value).getArray();
-            }
-
             // put a (ColumnLabel -> value) entry into the result map
             resultMap.put(metaData.getColumnLabel(i), value);
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8574d431/phoenix-pig/src/it/java/org/apache/phoenix/pig/PhoenixHBaseLoaderIT.java
----------------------------------------------------------------------
diff --git a/phoenix-pig/src/it/java/org/apache/phoenix/pig/PhoenixHBaseLoaderIT.java b/phoenix-pig/src/it/java/org/apache/phoenix/pig/PhoenixHBaseLoaderIT.java
index 606282a..5de323e 100644
--- a/phoenix-pig/src/it/java/org/apache/phoenix/pig/PhoenixHBaseLoaderIT.java
+++ b/phoenix-pig/src/it/java/org/apache/phoenix/pig/PhoenixHBaseLoaderIT.java
@@ -22,6 +22,7 @@ package org.apache.phoenix.pig;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
+import java.sql.Array;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.util.ArrayList;
@@ -39,6 +40,8 @@ import org.apache.pig.impl.logicalLayer.schema.Schema;
 import org.apache.pig.impl.logicalLayer.schema.Schema.FieldSchema;
 import org.junit.Test;
 
+import com.google.common.collect.Lists;
+
 /**
  * 
  * Test class to run all the integration tests against a virtual map reduce cluster.
@@ -672,4 +675,166 @@ public class PhoenixHBaseLoaderIT extends BasePigIT {
         List<Tuple> actualList = data.get("out");
         assertEquals(expectedList.size(), actualList.size());
     }
-}
+    
+   /**
+    * 
+    * @throws Exception
+    */
+    @Test
+    public void testLoadForArrayWithQuery() throws Exception {
+         //create the table
+        final String TABLE = "TABLE14";
+        String ddl = "CREATE TABLE  " + TABLE
+                + " ( ID INTEGER PRIMARY KEY, a_double_array double array[] , a_varchar_array varchar array, a_concat_str varchar, sep varchar)";
+                
+        conn.createStatement().execute(ddl);
+        
+        Double[] doubleArr =  new Double[3];
+        doubleArr[0] = 2.2;
+        doubleArr[1] = 4.4;
+        doubleArr[2] = 6.6;
+        Array doubleArray = conn.createArrayOf("DOUBLE", doubleArr);
+        Tuple doubleArrTuple = Storage.tuple(2.2d, 4.4d, 6.6d);
+        
+        Double[] doubleArr2 =  new Double[2];
+        doubleArr2[0] = 12.2;
+        doubleArr2[1] = 22.2;
+        Array doubleArray2 = conn.createArrayOf("DOUBLE", doubleArr2);
+        Tuple doubleArrTuple2 = Storage.tuple(12.2d, 22.2d);
+        
+        String[] strArr =  new String[4];
+        strArr[0] = "ABC";
+        strArr[1] = "DEF";
+        strArr[2] = "GHI";
+        strArr[3] = "JKL";
+        Array strArray  = conn.createArrayOf("VARCHAR", strArr);
+        Tuple strArrTuple = Storage.tuple("ABC", "DEF", "GHI", "JKL");
+        
+        String[] strArr2 =  new String[2];
+        strArr2[0] = "ABC";
+        strArr2[1] = "XYZ";
+        Array strArray2  = conn.createArrayOf("VARCHAR", strArr2);
+        Tuple strArrTuple2 = Storage.tuple("ABC", "XYZ");
+        
+        //upsert data.
+        final String dml = "UPSERT INTO " + TABLE + " VALUES(?, ?, ?, ?, ?) ";
+        PreparedStatement stmt = conn.prepareStatement(dml);
+        stmt.setInt(1, 1);
+        stmt.setArray(2, doubleArray);
+        stmt.setArray(3, strArray);
+        stmt.setString(4, "ONE,TWO,THREE");
+        stmt.setString(5, ",");
+        stmt.execute();
+        
+        stmt.setInt(1, 2);
+        stmt.setArray(2, doubleArray2);
+        stmt.setArray(3, strArray2);
+        stmt.setString(4, "FOUR:five:six");
+        stmt.setString(5, ":");
+        stmt.execute();
+       
+        conn.commit();
+        
+        Tuple dynArrTuple = Storage.tuple("ONE", "TWO", "THREE");
+        Tuple dynArrTuple2 = Storage.tuple("FOUR", "five", "six");
+        
+        //sql query
+        final String sqlQuery = String.format(" SELECT ID, A_DOUBLE_ARRAY, A_VARCHAR_ARRAY, REGEXP_SPLIT(a_concat_str, sep) AS flattend_str FROM %s ", TABLE); 
+      
+        final Data data = Storage.resetData(pigServer);
+        List<Tuple> expectedList = new ArrayList<Tuple>();
+        expectedList.add(Storage.tuple(1, 3L, 4L, dynArrTuple));
+        expectedList.add(Storage.tuple(2, 2L, 2L, dynArrTuple2));
+        final String load = String.format("A = load 'hbase://query/%s' using " + PhoenixHBaseLoader.class.getName() + "('%s');",sqlQuery,zkQuorum);
+        pigServer.setBatchOn();
+        pigServer.registerQuery(load);
+        pigServer.registerQuery("B = FOREACH A GENERATE ID, SIZE(A_DOUBLE_ARRAY), SIZE(A_VARCHAR_ARRAY), FLATTEND_STR;");
+        pigServer.registerQuery("STORE B INTO 'out' using mock.Storage();");
+        pigServer.executeBatch();
+        
+        List<Tuple> actualList = data.get("out");
+        assertEquals(expectedList.size(), actualList.size());
+        assertEquals(expectedList, actualList);
+        
+        Schema schema = pigServer.dumpSchema("A");
+        List<FieldSchema> fields = schema.getFields();
+        assertEquals(4, fields.size());
+        assertTrue(fields.get(0).alias.equalsIgnoreCase("ID"));
+        assertTrue(fields.get(0).type == DataType.INTEGER);
+        assertTrue(fields.get(1).alias.equalsIgnoreCase("A_DOUBLE_ARRAY"));
+        assertTrue(fields.get(1).type == DataType.TUPLE);
+        assertTrue(fields.get(2).alias.equalsIgnoreCase("A_VARCHAR_ARRAY"));
+        assertTrue(fields.get(2).type == DataType.TUPLE);
+        assertTrue(fields.get(3).alias.equalsIgnoreCase("FLATTEND_STR"));
+        assertTrue(fields.get(3).type == DataType.TUPLE);
+        
+        Iterator<Tuple> iterator = pigServer.openIterator("A");
+        Tuple firstTuple = Storage.tuple(1, doubleArrTuple, strArrTuple, dynArrTuple);
+        Tuple secondTuple = Storage.tuple(2, doubleArrTuple2, strArrTuple2, dynArrTuple2);
+        List<Tuple> expectedRows = Lists.newArrayList(firstTuple, secondTuple);
+        List<Tuple> actualRows = Lists.newArrayList();
+        while (iterator.hasNext()) {
+            Tuple tuple = iterator.next();
+            actualRows.add(tuple);
+        }
+        assertEquals(expectedRows, actualRows);
+    }
+    
+    
+    /**
+     * 
+     * @throws Exception
+     */
+     @Test
+     public void testLoadForArrayWithTable() throws Exception {
+          //create the table
+         final String TABLE = "TABLE15";
+         String ddl = "CREATE TABLE  " + TABLE
+                 + " ( ID INTEGER PRIMARY KEY, a_double_array double array[])";
+                 
+         conn.createStatement().execute(ddl);
+         
+         Double[] doubleArr =  new Double[3];
+         doubleArr[0] = 2.2;
+         doubleArr[1] = 4.4;
+         doubleArr[2] = 6.6;
+         Array doubleArray = conn.createArrayOf("DOUBLE", doubleArr);
+         Tuple doubleArrTuple = Storage.tuple(2.2d, 4.4d, 6.6d);
+         
+         Double[] doubleArr2 =  new Double[2];
+         doubleArr2[0] = 12.2;
+         doubleArr2[1] = 22.2;
+         Array doubleArray2 = conn.createArrayOf("DOUBLE", doubleArr2);
+         Tuple doubleArrTuple2 = Storage.tuple(12.2d, 22.2d);
+         
+         //upsert data.
+         final String dml = "UPSERT INTO " + TABLE + " VALUES(?, ?) ";
+         PreparedStatement stmt = conn.prepareStatement(dml);
+         stmt.setInt(1, 1);
+         stmt.setArray(2, doubleArray);
+         stmt.execute();
+         
+         stmt.setInt(1, 2);
+         stmt.setArray(2, doubleArray2);
+         stmt.execute();
+        
+         conn.commit();
+         
+         final Data data = Storage.resetData(pigServer);
+         List<Tuple> expectedList = new ArrayList<Tuple>();
+         expectedList.add(Storage.tuple(1, doubleArrTuple));
+         expectedList.add(Storage.tuple(2, doubleArrTuple2));
+         
+         pigServer.setBatchOn();
+         pigServer.registerQuery(String.format(
+             "A = load 'hbase://table/%s' using " + PhoenixHBaseLoader.class.getName() + "('%s');", TABLE,
+             zkQuorum));
+
+         pigServer.registerQuery("STORE A INTO 'out' using mock.Storage();");
+         pigServer.executeBatch();
+         
+         List<Tuple> actualList = data.get("out");
+         assertEquals(expectedList.size(), actualList.size());
+         assertEquals(expectedList, actualList);
+     }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8574d431/phoenix-pig/src/it/java/org/apache/phoenix/pig/PhoenixHBaseStorerIT.java
----------------------------------------------------------------------
diff --git a/phoenix-pig/src/it/java/org/apache/phoenix/pig/PhoenixHBaseStorerIT.java b/phoenix-pig/src/it/java/org/apache/phoenix/pig/PhoenixHBaseStorerIT.java
index 9e9434c..b9339bc 100644
--- a/phoenix-pig/src/it/java/org/apache/phoenix/pig/PhoenixHBaseStorerIT.java
+++ b/phoenix-pig/src/it/java/org/apache/phoenix/pig/PhoenixHBaseStorerIT.java
@@ -19,10 +19,12 @@
  */
 package org.apache.phoenix.pig;
 
+import static org.apache.pig.builtin.mock.Storage.tuple;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
+import java.sql.Array;
 import java.sql.ResultSet;
 import java.sql.Statement;
 import java.util.Collection;
@@ -254,4 +256,39 @@ public class PhoenixHBaseStorerIT extends BasePigIT {
         assertEquals(now, rs.getTimestamp(4).getTime());
      
     }
+    
+    @Test
+    public void testStoreForArray() throws Exception {
+     
+        final String tableName = "TABLE5";
+        final Statement stmt = conn.createStatement();
+        String ddl = "CREATE TABLE  " + tableName
+                + " ( ID INTEGER PRIMARY KEY, dbl double array[], a_varchar_array varchar array)";
+      
+        stmt.execute(ddl);
+      
+        final Data data = Storage.resetData(pigServer);
+        data.set("in",  tuple(1, tuple(2.2)),
+                        tuple(2, tuple(2.4, 2.5)),
+                        tuple(3, tuple(2.3)));
+
+        pigServer.setBatchOn();
+        pigServer.registerQuery("A = LOAD 'in' USING mock.Storage() as (id:int, dbl:tuple());");
+        pigServer.registerQuery("Store A into 'hbase://" + tableName + "/ID,DBL"
+                               + "' using " + PhoenixHBaseStorage.class.getName() + "('"
+                                + zkQuorum + "', '-batchSize 1000');");
+
+        if (pigServer.executeBatch().get(0).getStatus() != JOB_STATUS.COMPLETED) {
+             throw new RuntimeException("Job failed", pigServer.executeBatch()
+                    .get(0).getException());
+        }
+
+        final ResultSet rs = stmt
+                .executeQuery(String.format("SELECT id , dbl FROM %s where id = 2" , tableName));
+
+        assertTrue(rs.next());
+        assertEquals(2, rs.getInt(1));
+        Array expectedDoubleArr = conn.createArrayOf("DOUBLE", new Double[] { 2.4, 2.5 });
+        assertEquals(expectedDoubleArr,rs.getArray(2));
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8574d431/phoenix-pig/src/main/java/org/apache/phoenix/pig/PhoenixHBaseLoader.java
----------------------------------------------------------------------
diff --git a/phoenix-pig/src/main/java/org/apache/phoenix/pig/PhoenixHBaseLoader.java b/phoenix-pig/src/main/java/org/apache/phoenix/pig/PhoenixHBaseLoader.java
index adbb556..2be2317 100644
--- a/phoenix-pig/src/main/java/org/apache/phoenix/pig/PhoenixHBaseLoader.java
+++ b/phoenix-pig/src/main/java/org/apache/phoenix/pig/PhoenixHBaseLoader.java
@@ -29,20 +29,19 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.mapreduce.InputFormat;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.phoenix.mapreduce.PhoenixInputFormat;
+import org.apache.phoenix.mapreduce.PhoenixRecordWritable;
 import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil;
 import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil.SchemaType;
 import org.apache.phoenix.pig.util.PhoenixPigSchemaUtil;
 import org.apache.phoenix.pig.util.QuerySchemaParserFunction;
 import org.apache.phoenix.pig.util.TableSchemaParserFunction;
 import org.apache.phoenix.pig.util.TypeUtil;
-import org.apache.phoenix.pig.writable.PhoenixPigDBWritable;
 import org.apache.pig.Expression;
 import org.apache.pig.LoadFunc;
 import org.apache.pig.LoadMetadata;
@@ -57,6 +56,7 @@ import org.apache.pig.impl.util.UDFContext;
 
 import com.google.common.base.Preconditions;
 
+
 /**
  * LoadFunc to load data from HBase using Phoenix .
  * 
@@ -90,8 +90,8 @@ public final class PhoenixHBaseLoader extends LoadFunc implements LoadMetadata {
     private String tableName;
     private String selectQuery;
     private String zkQuorum ;
-    private PhoenixInputFormat<PhoenixPigDBWritable> inputFormat;
-    private RecordReader<NullWritable,PhoenixPigDBWritable> reader;
+    private PhoenixInputFormat<PhoenixRecordWritable> inputFormat;
+    private RecordReader<NullWritable,PhoenixRecordWritable> reader;
     private String contextSignature;
     private ResourceSchema schema;
        
@@ -122,13 +122,13 @@ public final class PhoenixHBaseLoader extends LoadFunc implements LoadMetadata {
      * @param configuration
      * @throws PigException
      */
-    private void initializePhoenixPigConfiguration(final String location, final Configuration configuration) throws PigException {
+    private void initializePhoenixPigConfiguration(final String location, final Configuration configuration) throws IOException {
         if(this.config != null) {
             return;
         }
         this.config = configuration;
         this.config.set(HConstants.ZOOKEEPER_QUORUM,this.zkQuorum);
-        PhoenixConfigurationUtil.setInputClass(this.config, PhoenixPigDBWritable.class);
+        PhoenixConfigurationUtil.setInputClass(this.config, PhoenixRecordWritable.class);
         Pair<String,String> pair = null;
         try {
             if (location.startsWith(PHOENIX_TABLE_NAME_SCHEME)) {
@@ -167,8 +167,8 @@ public final class PhoenixHBaseLoader extends LoadFunc implements LoadMetadata {
     @Override
     public InputFormat getInputFormat() throws IOException {
         if(inputFormat == null) {
-            inputFormat = new PhoenixInputFormat<PhoenixPigDBWritable>();
-            PhoenixConfigurationUtil.setInputClass(this.config,PhoenixPigDBWritable.class);
+            inputFormat = new PhoenixInputFormat<PhoenixRecordWritable>();
+            PhoenixConfigurationUtil.setInputClass(this.config, PhoenixRecordWritable.class);
         }
         return inputFormat;
     }
@@ -198,18 +198,18 @@ public final class PhoenixHBaseLoader extends LoadFunc implements LoadMetadata {
             if(!reader.nextKeyValue()) {
                 return null; 
              }
-             final PhoenixPigDBWritable record = reader.getCurrentValue();
+            final PhoenixRecordWritable record = reader.getCurrentValue();
             if(record == null) {
                 return null;
             }
-            final Tuple tuple = TypeUtil.transformToTuple(record,schema.getFields());
+            final Tuple tuple = TypeUtil.transformToTuple(record, schema.getFields());
             return tuple;
        } catch (InterruptedException e) {
             Thread.currentThread().interrupt();
             int errCode = 6018;
             final String errMsg = "Error while reading input";
             throw new ExecException(errMsg, errCode,PigException.REMOTE_ENVIRONMENT, e);
-        }
+       } 
     }
     
     private void printUsage(final String location) throws PigException {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8574d431/phoenix-pig/src/main/java/org/apache/phoenix/pig/PhoenixHBaseStorage.java
----------------------------------------------------------------------
diff --git a/phoenix-pig/src/main/java/org/apache/phoenix/pig/PhoenixHBaseStorage.java b/phoenix-pig/src/main/java/org/apache/phoenix/pig/PhoenixHBaseStorage.java
index 28060f3..a9f0c8f 100644
--- a/phoenix-pig/src/main/java/org/apache/phoenix/pig/PhoenixHBaseStorage.java
+++ b/phoenix-pig/src/main/java/org/apache/phoenix/pig/PhoenixHBaseStorage.java
@@ -37,16 +37,21 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.OutputFormat;
 import org.apache.hadoop.mapreduce.RecordWriter;
 import org.apache.phoenix.mapreduce.PhoenixOutputFormat;
+import org.apache.phoenix.mapreduce.PhoenixRecordWritable;
 import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil;
 import org.apache.phoenix.pig.util.TableSchemaParserFunction;
-import org.apache.phoenix.pig.writable.PhoenixPigDBWritable;
+import org.apache.phoenix.pig.util.TypeUtil;
+import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.util.ColumnInfo;
 import org.apache.pig.ResourceSchema;
 import org.apache.pig.ResourceSchema.ResourceFieldSchema;
 import org.apache.pig.StoreFuncInterface;
+import org.apache.pig.data.DataType;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.util.ObjectSerializer;
 import org.apache.pig.impl.util.UDFContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * StoreFunc that uses Phoenix to store data into HBase.
@@ -81,8 +86,10 @@ import org.apache.pig.impl.util.UDFContext;
 @SuppressWarnings("rawtypes")
 public class PhoenixHBaseStorage implements StoreFuncInterface {
 
+    private static final Logger LOG = LoggerFactory.getLogger(PhoenixHBaseStorage.class);
+    
     private Configuration config;
-    private RecordWriter<NullWritable, PhoenixPigDBWritable> writer;
+    private RecordWriter<NullWritable, PhoenixRecordWritable> writer;
     private List<ColumnInfo> columnInfo = null;
     private String contextSignature = null;
     private ResourceSchema schema;  
@@ -167,16 +174,28 @@ public class PhoenixHBaseStorage implements StoreFuncInterface {
 
     @Override
     public void putNext(Tuple t) throws IOException {
-        ResourceFieldSchema[] fieldSchemas = (schema == null) ? null : schema.getFields();      
-        PhoenixPigDBWritable record = PhoenixPigDBWritable.newInstance(fieldSchemas,this.columnInfo);
-        for(int i=0; i<t.size(); i++) {
-            record.add(t.get(i));
-        }
+        ResourceFieldSchema[] fieldSchemas = (schema == null) ? null : schema.getFields();
+        PhoenixRecordWritable record = new PhoenixRecordWritable(this.columnInfo);
         try {
+            for(int i=0; i<t.size(); i++) {
+                Object value = t.get(i);
+                if(value == null) {
+                    record.add(null);
+                    continue;
+                }
+                ColumnInfo cinfo = this.columnInfo.get(i);
+                byte type = (fieldSchemas == null) ? DataType.findType(value) : fieldSchemas[i].getType();
+                PDataType pDataType = PDataType.fromTypeId(cinfo.getSqlType());
+                Object v =  TypeUtil.castPigTypeToPhoenix(value, type, pDataType);
+                record.add(v);
+            }
             this.writer.write(null, record);
         } catch (InterruptedException e) {
             Thread.currentThread().interrupt();
             throw new RuntimeException(e);
+        } catch (SQLException e) {
+            LOG.error("Error on tuple {} .",t);
+            throw new IOException(e);
         }
         
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8574d431/phoenix-pig/src/main/java/org/apache/phoenix/pig/util/TypeUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-pig/src/main/java/org/apache/phoenix/pig/util/TypeUtil.java b/phoenix-pig/src/main/java/org/apache/phoenix/pig/util/TypeUtil.java
index 53d3ae3..8c9bd6a 100644
--- a/phoenix-pig/src/main/java/org/apache/phoenix/pig/util/TypeUtil.java
+++ b/phoenix-pig/src/main/java/org/apache/phoenix/pig/util/TypeUtil.java
@@ -20,14 +20,17 @@ package org.apache.phoenix.pig.util;
 
 import java.io.IOException;
 import java.sql.Date;
+import java.sql.SQLException;
 import java.sql.Time;
 import java.sql.Timestamp;
 import java.sql.Types;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.phoenix.pig.writable.PhoenixPigDBWritable;
+import org.apache.phoenix.mapreduce.PhoenixRecordWritable;
+import org.apache.phoenix.schema.types.PArrayDataType;
 import org.apache.phoenix.schema.types.PBinary;
 import org.apache.phoenix.schema.types.PBoolean;
 import org.apache.phoenix.schema.types.PChar;
@@ -53,6 +56,7 @@ import org.apache.phoenix.schema.types.PUnsignedTimestamp;
 import org.apache.phoenix.schema.types.PUnsignedTinyint;
 import org.apache.phoenix.schema.types.PVarbinary;
 import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.schema.types.PhoenixArray;
 import org.apache.pig.PigException;
 import org.apache.pig.ResourceSchema.ResourceFieldSchema;
 import org.apache.pig.backend.hadoop.hbase.HBaseBinaryConverter;
@@ -72,7 +76,7 @@ public final class TypeUtil {
     private static final HBaseBinaryConverter BINARY_CONVERTER = new HBaseBinaryConverter();
     private static final ImmutableMap<PDataType, Byte> PHOENIX_TO_PIG_TYPE = init();
     private static final TupleFactory TUPLE_FACTORY = TupleFactory.getInstance();
-
+    
     private TypeUtil() {}
 
     /**
@@ -157,15 +161,24 @@ public final class TypeUtil {
     }
 
     /**
-     * This method encodes a value with Phoenix data type. It begins with checking whether an object is BINARY and makes
+     * This method encodes a value with Phoenix data type. It begins with checking whether an object is TUPLE. A {@link Tuple} is mapped
+     * to a {@link PArrayDataType} .  It then checks if it is BINARY and makes
      * a call to {@link #castBytes(Object, PDataType)} to convert bytes to targetPhoenixType. It returns a {@link RuntimeException}
      * when object can not be coerced.
      * 
      * @param o
      * @param targetPhoenixType
      * @return Object
+     * @throws SQLException 
      */
-    public static Object castPigTypeToPhoenix(Object o, byte objectType, PDataType targetPhoenixType) {
+    public static Object castPigTypeToPhoenix(Object o, byte objectType, PDataType targetPhoenixType) throws SQLException {
+        
+        if(DataType.TUPLE == objectType) {
+            Tuple tuple = (Tuple)o;
+            List<Object> data = tuple.getAll();
+            return data.toArray();
+        }
+        
         PDataType inferredPType = getType(o, objectType);
 
         if (inferredPType == null) { return null; }
@@ -237,20 +250,22 @@ public final class TypeUtil {
      * @return
      * @throws IOException
      */
-    public static Tuple transformToTuple(final PhoenixPigDBWritable record, final ResourceFieldSchema[] projectedColumns)
+    public static Tuple transformToTuple(final PhoenixRecordWritable record, final ResourceFieldSchema[] projectedColumns) 
             throws IOException {
 
-        List<Object> columnValues = record.getValues();
+        Map<String, Object> columnValues = record.getResultMap();
+        
         if (columnValues == null || columnValues.size() == 0 || projectedColumns == null
                 || projectedColumns.length != columnValues.size()) { return null; }
         int numColumns = columnValues.size();
         Tuple tuple = TUPLE_FACTORY.newTuple(numColumns);
         try {
-            for (int i = 0; i < numColumns; i++) {
+            int i = 0;
+            for (Map.Entry<String,Object> entry : columnValues.entrySet()) {
                 final ResourceFieldSchema fieldSchema = projectedColumns[i];
-                Object object = columnValues.get(i);
+                Object object = entry.getValue();
                 if (object == null) {
-                    tuple.set(i, null);
+                    tuple.set(i++, null);
                     continue;
                 }
 
@@ -289,9 +304,20 @@ public final class TypeUtil {
                 case DataType.BIGINTEGER:
                     tuple.set(i, DataType.toBigInteger(object));
                     break;
+                case DataType.TUPLE:
+                {
+                    PhoenixArray array = (PhoenixArray)object;
+                    Tuple t = TUPLE_FACTORY.newTuple(array.getDimensions());;
+                    for(int j = 0 ; j < array.getDimensions() ; j++) {
+                        t.set(j,array.getElement(j));
+                    }
+                    tuple.set(i, t);
+                    break;
+                }
                 default:
                     throw new RuntimeException(String.format(" Not supported [%s] pig type", fieldSchema));
                 }
+                i++;
             }
         } catch (Exception ex) {
             final String errorMsg = String.format(" Error transforming PhoenixRecord to Tuple [%s] ", ex.getMessage());
@@ -300,7 +326,7 @@ public final class TypeUtil {
         }
         return tuple;
     }
-
+    
     /**
      * Returns the mapping pig data type for a given phoenix data type.
      * 
@@ -309,6 +335,9 @@ public final class TypeUtil {
      */
     public static Byte getPigDataTypeForPhoenixType(final PDataType phoenixDataType) {
         Preconditions.checkNotNull(phoenixDataType);
+        if(phoenixDataType instanceof PArrayDataType) {
+            return DataType.TUPLE;
+        }
         final Byte pigDataType = PHOENIX_TO_PIG_TYPE.get(phoenixDataType);
         if (LOG.isDebugEnabled()) {
             LOG.debug(String.format(" For PhoenixDataType [%s] , pigDataType is [%s] ",

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8574d431/phoenix-pig/src/main/java/org/apache/phoenix/pig/writable/PhoenixPigDBWritable.java
----------------------------------------------------------------------
diff --git a/phoenix-pig/src/main/java/org/apache/phoenix/pig/writable/PhoenixPigDBWritable.java b/phoenix-pig/src/main/java/org/apache/phoenix/pig/writable/PhoenixPigDBWritable.java
deleted file mode 100644
index 566e427..0000000
--- a/phoenix-pig/src/main/java/org/apache/phoenix/pig/writable/PhoenixPigDBWritable.java
+++ /dev/null
@@ -1,121 +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.phoenix.pig.writable;
-
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.mapreduce.lib.db.DBWritable;
-import org.apache.phoenix.pig.util.TypeUtil;
-import org.apache.phoenix.schema.types.PDataType;
-import org.apache.phoenix.util.ColumnInfo;
-import org.apache.pig.ResourceSchema.ResourceFieldSchema;
-import org.apache.pig.data.DataType;
-
-import com.google.common.base.Preconditions;
-
-/**
- * A {@link Writable} representing a Phoenix record. This class
- * a) does a type mapping and sets the value accordingly in the {@link PreparedStatement}
- * b) reads the column values from the {@link ResultSet}
- * 
- */
-public class PhoenixPigDBWritable implements DBWritable {
-    
-    private final List<Object> values;
-    private ResourceFieldSchema[] fieldSchemas;
-    private List<ColumnInfo> columnMetadataList;
-  
-    public PhoenixPigDBWritable() {
-        this.values = new ArrayList<Object>();
-    }
-    
-    @Override
-    public void write(PreparedStatement statement) throws SQLException {
-        for (int i = 0; i < columnMetadataList.size(); i++) {
-            Object o = values.get(i);
-            ColumnInfo columnInfo = columnMetadataList.get(i);
-            byte type = (fieldSchemas == null) ? DataType.findType(o) : fieldSchemas[i].getType();
-            try {
-                Object upsertValue = convertTypeSpecificValue(o, type, columnInfo.getSqlType());
-                if (upsertValue != null) {
-                    statement.setObject(i + 1, upsertValue, columnInfo.getSqlType());
-                } else {
-                    statement.setNull(i + 1, columnInfo.getSqlType());
-                }
-            } catch (RuntimeException re) {
-                throw new RuntimeException(String.format("Unable to process column %s, innerMessage=%s"
-                        ,columnInfo.toString(),re.getMessage()),re);
-                
-            }
-        }
-    }
-    
-    public void add(Object value) {
-        values.add(value);
-    }
-
-    private Object convertTypeSpecificValue(Object o, byte type, Integer sqlType) {
-        PDataType pDataType = PDataType.fromTypeId(sqlType);
-        return TypeUtil.castPigTypeToPhoenix(o, type, pDataType);
-    }
-
-    public List<Object> getValues() {
-        return values;
-    }
-
-    @Override
-    public void readFields(final ResultSet rs) throws SQLException {
-        Preconditions.checkNotNull(rs);
-        final int noOfColumns = rs.getMetaData().getColumnCount();
-        values.clear();
-        for(int i = 1 ; i <= noOfColumns ; i++) {
-            Object obj = rs.getObject(i);
-            values.add(obj);
-        }
-    }
-
-    public ResourceFieldSchema[] getFieldSchemas() {
-        return fieldSchemas;
-    }
-
-    public void setFieldSchemas(ResourceFieldSchema[] fieldSchemas) {
-        this.fieldSchemas = fieldSchemas;
-    }
-
-    public List<ColumnInfo> getColumnMetadataList() {
-        return columnMetadataList;
-    }
-
-    public void setColumnMetadataList(List<ColumnInfo> columnMetadataList) {
-        this.columnMetadataList = columnMetadataList;
-    }
-
-    public static PhoenixPigDBWritable newInstance(final ResourceFieldSchema[] fieldSchemas,
-            final List<ColumnInfo> columnMetadataList) {
-        final PhoenixPigDBWritable dbWritable = new PhoenixPigDBWritable ();
-        dbWritable.setFieldSchemas(fieldSchemas);
-        dbWritable.setColumnMetadataList(columnMetadataList);
-        return dbWritable;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8574d431/phoenix-pig/src/test/java/org/apache/phoenix/pig/util/TypeUtilTest.java
----------------------------------------------------------------------
diff --git a/phoenix-pig/src/test/java/org/apache/phoenix/pig/util/TypeUtilTest.java b/phoenix-pig/src/test/java/org/apache/phoenix/pig/util/TypeUtilTest.java
index 0b44d2b..e459dc1 100644
--- a/phoenix-pig/src/test/java/org/apache/phoenix/pig/util/TypeUtilTest.java
+++ b/phoenix-pig/src/test/java/org/apache/phoenix/pig/util/TypeUtilTest.java
@@ -24,44 +24,57 @@ import static org.mockito.Mockito.when;
 
 import java.math.BigDecimal;
 import java.math.BigInteger;
-import java.util.List;
+import java.util.Map;
 
-import org.apache.phoenix.pig.writable.PhoenixPigDBWritable;
+import org.apache.phoenix.mapreduce.PhoenixRecordWritable;
+import org.apache.phoenix.schema.types.PArrayDataType;
+import org.apache.phoenix.schema.types.PDouble;
+import org.apache.phoenix.schema.types.PhoenixArray;
 import org.apache.pig.ResourceSchema.ResourceFieldSchema;
 import org.apache.pig.data.DataType;
 import org.apache.pig.data.Tuple;
 import org.junit.Test;
 
-import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 
 public class TypeUtilTest {
 
     @Test
     public void testTransformToTuple() throws Exception {
-        PhoenixPigDBWritable record = mock(PhoenixPigDBWritable.class);
-        List<Object> values = Lists.newArrayList();
-        values.add("213123");
-        values.add(1231123);
-        values.add(31231231232131L);
-        values.add("bytearray".getBytes());
-        when(record.getValues()).thenReturn(values);
+        PhoenixRecordWritable record = mock(PhoenixRecordWritable.class);
+        Double[] doubleArr =  new Double[2];
+        doubleArr[0] = 64.87;
+        doubleArr[1] = 89.96;
+        PhoenixArray arr = PArrayDataType.instantiatePhoenixArray(PDouble.INSTANCE, doubleArr);
+        Map<String,Object> values = Maps.newLinkedHashMap();
+        values.put("first", "213123");
+        values.put("second", 1231123);
+        values.put("third", 31231231232131L);
+        values.put("four", "bytearray".getBytes());
+        values.put("five", arr);
+        when(record.getResultMap()).thenReturn(values);
 
         ResourceFieldSchema field = new ResourceFieldSchema().setType(DataType.CHARARRAY);
         ResourceFieldSchema field1 = new ResourceFieldSchema().setType(DataType.INTEGER);
         ResourceFieldSchema field2 = new ResourceFieldSchema().setType(DataType.LONG);
         ResourceFieldSchema field3 = new ResourceFieldSchema().setType(DataType.BYTEARRAY);
-        ResourceFieldSchema[] projectedColumns = { field, field1, field2, field3 };
+        ResourceFieldSchema field4 = new ResourceFieldSchema().setType(DataType.TUPLE);
+        ResourceFieldSchema[] projectedColumns = { field, field1, field2, field3 , field4 }; 
 
         Tuple t = TypeUtil.transformToTuple(record, projectedColumns);
 
         assertEquals(DataType.LONG, DataType.findType(t.get(2)));
+        assertEquals(DataType.TUPLE, DataType.findType(t.get(4)));
+        Tuple doubleArrayTuple = (Tuple)t.get(4);
+        assertEquals(2,doubleArrayTuple.size());
 
         field = new ResourceFieldSchema().setType(DataType.BIGDECIMAL);
         field1 = new ResourceFieldSchema().setType(DataType.BIGINTEGER);
         values.clear();
-        values.add(new BigDecimal(123123123.123213));
-        values.add(new BigInteger("1312313231312"));
+        values.put("first", new BigDecimal(123123123.123213));
+        values.put("second", new BigInteger("1312313231312"));
         ResourceFieldSchema[] columns = { field, field1 };
+        
         t = TypeUtil.transformToTuple(record, columns);
 
         assertEquals(DataType.BIGDECIMAL, DataType.findType(t.get(0)));


[07/50] [abbrv] phoenix git commit: PHOENIX-2478 Rows committed in transaction overlapping index creation are not populated

Posted by ma...@apache.org.
PHOENIX-2478 Rows committed in transaction overlapping index creation are not populated


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

Branch: refs/heads/calcite
Commit: 1717f12d7c6e9b1ee8ddffb55cea929d6d8543f1
Parents: 3520e12
Author: James Taylor <jt...@salesforce.com>
Authored: Tue Jan 19 13:16:50 2016 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Tue Jan 19 13:16:50 2016 -0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/phoenix/execute/MutationState.java  | 4 ----
 1 file changed, 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/1717f12d/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
index 8caac5d..d3b36ec 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
@@ -1231,10 +1231,6 @@ public class MutationState implements SQLCloseable {
                 throw new TableNotFoundException(dataTable.getSchemaName().getString(), dataTable.getTableName().getString());
             }
             tableRef.setTable(result.getTable());
-            if (!result.wasUpdated()) {
-                if (logger.isInfoEnabled()) logger.info("No updates to " + dataTable.getName().getString() + " as of "  + timestamp);
-                continue;
-            }
             if (!addedIndexes) {
                 // TODO: in theory we should do a deep equals check here, as it's possible
                 // that an index was dropped and recreated with the same name but different


[49/50] [abbrv] phoenix git commit: Sync with master; Bug fix for ResultSet object of Float type; Walkaround for PHOENIX-2647

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/74409be8/phoenix-core/src/it/java/org/apache/phoenix/calcite/CalciteIT.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/it/java/org/apache/phoenix/calcite/CalciteIT.java
index acd040d,0000000..41d0542
mode 100644,000000..100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/calcite/CalciteIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/calcite/CalciteIT.java
@@@ -1,1206 -1,0 +1,1206 @@@
 +/*
 + * 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.phoenix.calcite;
 +
 +import static org.apache.phoenix.util.TestUtil.JOIN_CUSTOMER_TABLE_FULL_NAME;
 +import static org.apache.phoenix.util.TestUtil.JOIN_ITEM_TABLE_FULL_NAME;
 +import static org.apache.phoenix.util.TestUtil.JOIN_ORDER_TABLE_FULL_NAME;
 +import static org.apache.phoenix.util.TestUtil.JOIN_SUPPLIER_TABLE_FULL_NAME;
 +
 +import java.sql.Connection;
 +import java.sql.DriverManager;
 +import java.sql.Timestamp;
 +import java.text.SimpleDateFormat;
 +import java.util.Properties;
 +
 +import org.junit.Before;
 +import org.junit.Ignore;
 +import org.junit.Test;
 +
 +/**
 + * Integration test for queries powered by Calcite.
 + */
 +public class CalciteIT extends BaseCalciteIT {
 +    
 +    @Before
 +    public void initTable() throws Exception {
 +        final String url = getUrl();
 +        initATableValues(getOrganizationId(), null, url);
 +        initJoinTableValues(url, null, null);
 +        initArrayTable();
 +        initSaltedTables(null);
 +        final Connection connection = DriverManager.getConnection(url);
 +        connection.createStatement().execute("CREATE VIEW IF NOT EXISTS v AS SELECT * from aTable where a_string = 'a'");
 +        connection.createStatement().execute("CREATE SEQUENCE IF NOT EXISTS seq0 START WITH 1 INCREMENT BY 1");
 +        connection.createStatement().execute("CREATE SEQUENCE IF NOT EXISTS my.seq1 START WITH 2 INCREMENT BY 2");
 +        connection.createStatement().execute("UPDATE STATISTICS ATABLE");
 +        connection.createStatement().execute("UPDATE STATISTICS " + JOIN_CUSTOMER_TABLE_FULL_NAME);
 +        connection.createStatement().execute("UPDATE STATISTICS " + JOIN_ITEM_TABLE_FULL_NAME);
 +        connection.createStatement().execute("UPDATE STATISTICS " + JOIN_SUPPLIER_TABLE_FULL_NAME);
 +        connection.createStatement().execute("UPDATE STATISTICS " + JOIN_ORDER_TABLE_FULL_NAME);
 +        connection.createStatement().execute("UPDATE STATISTICS " + SCORES_TABLE_NAME);
 +        connection.createStatement().execute("UPDATE STATISTICS " + SALTED_TABLE_NAME);
 +        connection.close();
 +    }
 +    
 +    private SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
 +    
 +    @Test public void testTableScan() throws Exception {
 +        start(false).sql("select * from aTable where a_string = 'a'")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixTableScan(table=[[phoenix, ATABLE]], filter=[=($2, 'a')])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                          {"00D300000000XHP", "00A123122312312", "a"}, 
 +                          {"00D300000000XHP", "00A223122312312", "a"}, 
 +                          {"00D300000000XHP", "00A323122312312", "a"}, 
 +                          {"00D300000000XHP", "00A423122312312", "a"}})
 +                .close();
 +        
 +        // FIXME: Should be 14:22:56 instead. Wrong due to time zone.
 +        start(false).sql("select \"DATE\" from " + JOIN_ORDER_TABLE_FULL_NAME + " where \"order_id\" = '000000000000001'")
-                 .resultIs(new Object[][]{
++                .resultIs(false, new Object[][]{
 +                        {new Timestamp(format.parse("2013-11-22 19:22:56").getTime())}})
 +                .close();
 +        
 +        start(false).sql("select student_id, scores from " + SCORES_TABLE_NAME)
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                        {1, new Integer[] {85, 80, 82}},
 +                        {2, null},
 +                        {3, new Integer[] {87, 88, 80}}})
 +                .close();
 +    }
 +    
 +    @Test public void testProject() throws Exception {
 +        start(false).sql("select entity_id, a_string, organization_id from aTable where a_string = 'a'")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixServerProject(ENTITY_ID=[$1], A_STRING=[$2], ORGANIZATION_ID=[$0])\n" +
 +                           "    PhoenixTableScan(table=[[phoenix, ATABLE]], filter=[=($2, 'a')])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                          {"00A123122312312", "a", "00D300000000XHP"}, 
 +                          {"00A223122312312", "a", "00D300000000XHP"}, 
 +                          {"00A323122312312", "a", "00D300000000XHP"}, 
 +                          {"00A423122312312", "a", "00D300000000XHP"}})
 +                .close();
 +    }
 +    
 +    @Test public void testJoin() throws Exception {
 +        start(false).sql("select t1.entity_id, t2.a_string, t1.organization_id from aTable t1 join aTable t2 on t1.entity_id = t2.entity_id and t1.organization_id = t2.organization_id where t1.a_string = 'a'") 
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixClientProject(ENTITY_ID=[$4], A_STRING=[$2], ORGANIZATION_ID=[$3])\n" +
 +                           "    PhoenixServerJoin(condition=[AND(=($4, $1), =($3, $0))], joinType=[inner])\n" +
 +                           "      PhoenixServerProject(ORGANIZATION_ID=[$0], ENTITY_ID=[$1], A_STRING=[$2])\n" +
 +                           "        PhoenixTableScan(table=[[phoenix, ATABLE]])\n" +
 +                           "      PhoenixServerProject(ORGANIZATION_ID=[$0], ENTITY_ID=[$1], A_STRING=[$2])\n" +
 +                           "        PhoenixTableScan(table=[[phoenix, ATABLE]], filter=[=($2, 'a')])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                          {"00A123122312312", "a", "00D300000000XHP"}, 
 +                          {"00A223122312312", "a", "00D300000000XHP"}, 
 +                          {"00A323122312312", "a", "00D300000000XHP"}, 
 +                          {"00A423122312312", "a", "00D300000000XHP"}})
 +                .close();
 +        
 +        start(false).sql("SELECT item.\"item_id\", item.name, supp.\"supplier_id\", supp.name FROM " + JOIN_ITEM_TABLE_FULL_NAME + " item JOIN " + JOIN_SUPPLIER_TABLE_FULL_NAME + " supp ON item.\"supplier_id\" = supp.\"supplier_id\"")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixClientProject(item_id=[$0], NAME=[$1], supplier_id=[$3], NAME0=[$4])\n" +
 +                           "    PhoenixServerJoin(condition=[=($2, $3)], joinType=[inner])\n" +
 +                           "      PhoenixServerProject(item_id=[$0], NAME=[$1], supplier_id=[$5])\n" +
 +                           "        PhoenixTableScan(table=[[phoenix, Join, ItemTable]])\n" +
 +                           "      PhoenixServerProject(supplier_id=[$0], NAME=[$1])\n" +
 +                           "        PhoenixTableScan(table=[[phoenix, Join, SupplierTable]])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                          {"0000000001", "T1", "0000000001", "S1"}, 
 +                          {"0000000002", "T2", "0000000001", "S1"}, 
 +                          {"0000000003", "T3", "0000000002", "S2"}, 
 +                          {"0000000004", "T4", "0000000002", "S2"},
 +                          {"0000000005", "T5", "0000000005", "S5"},
 +                          {"0000000006", "T6", "0000000006", "S6"}})
 +                .close();
 +        
 +        start(false).sql("SELECT * FROM " + JOIN_ITEM_TABLE_FULL_NAME + " item JOIN " + JOIN_SUPPLIER_TABLE_FULL_NAME + " supp ON item.\"supplier_id\" = supp.\"supplier_id\" AND supp.name = 'S5'")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixClientProject(item_id=[$0], NAME=[$1], PRICE=[$2], DISCOUNT1=[$3], DISCOUNT2=[$4], supplier_id=[$5], DESCRIPTION=[$6], supplier_id0=[$7], NAME0=[$8], PHONE=[$9], ADDRESS=[$10], LOC_ID=[$11])\n" +
 +                           "    PhoenixServerJoin(condition=[=($5, $7)], joinType=[inner])\n" +
 +                           "      PhoenixTableScan(table=[[phoenix, Join, ItemTable]])\n" +
 +                           "      PhoenixServerProject(supplier_id=[$0], NAME=[$1], PHONE=[$2], ADDRESS=[$3], LOC_ID=[$4], NAME5=[CAST($1):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"])\n" +
 +                           "        PhoenixTableScan(table=[[phoenix, Join, SupplierTable]], filter=[=(CAST($1):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\", 'S5')])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                          {"0000000005", "T5", 500, 8, 15, "0000000005", "Item T5", "0000000005", "S5", "888-888-5555", "505 YYY Street", "10005"}})
 +                .close();
 +        
 +        start(false).sql("SELECT \"order_id\", i.name, i.price, discount2, quantity FROM " + JOIN_ORDER_TABLE_FULL_NAME + " o INNER JOIN " 
 +                + JOIN_ITEM_TABLE_FULL_NAME + " i ON o.\"item_id\" = i.\"item_id\" AND o.price = (i.price * (100 - discount2)) / 100.0 WHERE quantity < 5000")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixClientProject(order_id=[$5], NAME=[$1], PRICE=[$2], DISCOUNT2=[$3], QUANTITY=[$7])\n" +
 +                           "    PhoenixServerJoin(condition=[AND(=($6, $0), =($8, $4))], joinType=[inner])\n" +
 +                           "      PhoenixServerProject(item_id=[$0], NAME=[$1], PRICE=[$2], DISCOUNT2=[$4], $f7=[/(*($2, -(100, $4)), 100.0)])\n" +
 +                           "        PhoenixTableScan(table=[[phoenix, Join, ItemTable]])\n" +
 +                           "      PhoenixServerProject(order_id=[$0], item_id=[$2], QUANTITY=[$4], PRICE7=[CAST($3):DECIMAL(17, 6)])\n" +
 +                           "        PhoenixTableScan(table=[[phoenix, Join, OrderTable]], filter=[<($4, 5000)])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                          {"000000000000004", "T6", 600, 15, 4000}})
 +                .close();
 +    }
 +    
 +    @Test public void testRightOuterJoin() throws Exception {
 +        start(false).sql("SELECT item.\"item_id\", item.name, supp.\"supplier_id\", supp.name FROM " + JOIN_ITEM_TABLE_FULL_NAME + " item RIGHT OUTER JOIN " + JOIN_SUPPLIER_TABLE_FULL_NAME + " supp ON item.\"supplier_id\" = supp.\"supplier_id\"")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixClientProject(item_id=[$2], NAME=[$3], supplier_id=[$0], NAME0=[$1])\n" +
 +                           "    PhoenixServerJoin(condition=[=($4, $0)], joinType=[left])\n" +
 +                           "      PhoenixServerProject(supplier_id=[$0], NAME=[$1])\n" +
 +                           "        PhoenixTableScan(table=[[phoenix, Join, SupplierTable]])\n" +
 +                           "      PhoenixServerProject(item_id=[$0], NAME=[$1], supplier_id=[$5])\n" +
 +                           "        PhoenixTableScan(table=[[phoenix, Join, ItemTable]])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                          {"0000000001", "T1", "0000000001", "S1"}, 
 +                          {"0000000002", "T2", "0000000001", "S1"}, 
 +                          {"0000000003", "T3", "0000000002", "S2"}, 
 +                          {"0000000004", "T4", "0000000002", "S2"},
 +                          {null, null, "0000000003", "S3"}, 
 +                          {null, null, "0000000004", "S4"}, 
 +                          {"0000000005", "T5", "0000000005", "S5"},
 +                          {"0000000006", "T6", "0000000006", "S6"}})
 +                .close();
 +    }
 +    
 +    @Test public void testClientJoin() throws Exception {        
 +        start(false).sql("SELECT item.\"item_id\", item.name, supp.\"supplier_id\", supp.name FROM " + JOIN_ITEM_TABLE_FULL_NAME + " item FULL OUTER JOIN " + JOIN_SUPPLIER_TABLE_FULL_NAME + " supp ON item.\"supplier_id\" = supp.\"supplier_id\" order by \"item_id\", supp.name")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixClientSort(sort0=[$0], sort1=[$3], dir0=[ASC], dir1=[ASC])\n" +
 +                           "    PhoenixClientProject(item_id=[$0], NAME=[$1], supplier_id=[$3], NAME0=[$4])\n" +
 +                           "      PhoenixClientJoin(condition=[=($2, $3)], joinType=[full])\n" +
 +                           "        PhoenixServerSort(sort0=[$2], dir0=[ASC])\n" +
 +                           "          PhoenixServerProject(item_id=[$0], NAME=[$1], supplier_id=[$5])\n" +
 +                           "            PhoenixTableScan(table=[[phoenix, Join, ItemTable]])\n" +
 +                           "        PhoenixServerProject(supplier_id=[$0], NAME=[$1])\n" +
 +                           "          PhoenixTableScan(table=[[phoenix, Join, SupplierTable]], scanOrder=[FORWARD])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(true, new Object[][] {
 +                        {"0000000001", "T1", "0000000001", "S1"},
 +                        {"0000000002", "T2", "0000000001", "S1"},
 +                        {"0000000003", "T3", "0000000002", "S2"},
 +                        {"0000000004", "T4", "0000000002", "S2"},
 +                        {"0000000005", "T5", "0000000005", "S5"},
 +                        {"0000000006", "T6", "0000000006", "S6"},
 +                        {"invalid001", "INVALID-1", null, null},
 +                        {null, null, "0000000003", "S3"},
 +                        {null, null, "0000000004", "S4"}})
 +                .close();
 +        
 +        start(false).sql("select t1.entity_id, t2.a_string, t1.organization_id from aTable t1 join aTable t2 on t1.organization_id = t2.organization_id and t1.entity_id = t2.entity_id")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixClientProject(ENTITY_ID=[$1], A_STRING=[$4], ORGANIZATION_ID=[$0])\n" +
 +                           "    PhoenixClientJoin(condition=[AND(=($0, $2), =($1, $3))], joinType=[inner])\n" +
 +                           "      PhoenixServerProject(ORGANIZATION_ID=[$0], ENTITY_ID=[$1])\n" +
 +                           "        PhoenixTableScan(table=[[phoenix, ATABLE]], scanOrder=[FORWARD])\n" +
 +                           "      PhoenixServerProject(ORGANIZATION_ID=[$0], ENTITY_ID=[$1], A_STRING=[$2])\n" +
 +                           "        PhoenixTableScan(table=[[phoenix, ATABLE]], scanOrder=[FORWARD])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                          {"00A123122312312", "a", "00D300000000XHP"},
 +                          {"00A223122312312", "a", "00D300000000XHP"},
 +                          {"00A323122312312", "a", "00D300000000XHP"},
 +                          {"00A423122312312", "a", "00D300000000XHP"},
 +                          {"00B523122312312", "b", "00D300000000XHP"},
 +                          {"00B623122312312", "b", "00D300000000XHP"},
 +                          {"00B723122312312", "b", "00D300000000XHP"},
 +                          {"00B823122312312", "b", "00D300000000XHP"},
 +                          {"00C923122312312", "c", "00D300000000XHP"}})
 +                .close();
 +    }
 +    
 +    @Test public void testJoinPlanningWithCollation() throws Exception { 
 +        // Server-join with LHS sorted on order-by fields
 +        start(false).sql("SELECT item.\"item_id\", item.name, supp.\"supplier_id\", supp.name FROM " + JOIN_ITEM_TABLE_FULL_NAME + " item JOIN " + JOIN_SUPPLIER_TABLE_FULL_NAME + " supp ON item.\"supplier_id\" = supp.\"supplier_id\" order by supp.\"supplier_id\"")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixClientProject(item_id=[$2], NAME=[$3], supplier_id=[$0], NAME0=[$1])\n" +
 +                           "    PhoenixServerJoin(condition=[=($4, $0)], joinType=[inner])\n" +
 +                           "      PhoenixServerProject(supplier_id=[$0], NAME=[$1])\n" +
 +                           "        PhoenixTableScan(table=[[phoenix, Join, SupplierTable]], scanOrder=[FORWARD])\n" +
 +                           "      PhoenixServerProject(item_id=[$0], NAME=[$1], supplier_id=[$5])\n" +
 +                           "        PhoenixTableScan(table=[[phoenix, Join, ItemTable]])\n")
 +                .close();
 +        
 +        // Server-join with LHS reversely sorted on order-by fields
 +        start(false).sql("SELECT item.\"item_id\", item.name, supp.\"supplier_id\", supp.name FROM " + JOIN_ITEM_TABLE_FULL_NAME + " item JOIN " + JOIN_SUPPLIER_TABLE_FULL_NAME + " supp ON item.\"supplier_id\" = supp.\"supplier_id\" order by supp.\"supplier_id\" DESC")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixClientProject(item_id=[$2], NAME=[$3], supplier_id=[$0], NAME0=[$1])\n" +
 +                           "    PhoenixServerJoin(condition=[=($4, $0)], joinType=[inner])\n" +
 +                           "      PhoenixServerProject(supplier_id=[$0], NAME=[$1])\n" +
 +                           "        PhoenixTableScan(table=[[phoenix, Join, SupplierTable]], scanOrder=[REVERSE])\n" +
 +                           "      PhoenixServerProject(item_id=[$0], NAME=[$1], supplier_id=[$5])\n" +
 +                           "        PhoenixTableScan(table=[[phoenix, Join, ItemTable]])\n")
 +                .close();
 +        
 +        // Join key being order-by fields with the other side sorted on order-by fields
 +        start(false).sql("SELECT item.\"item_id\", item.name, supp.\"supplier_id\", supp.name FROM " + JOIN_ITEM_TABLE_FULL_NAME + " item JOIN " + JOIN_SUPPLIER_TABLE_FULL_NAME + " supp ON item.\"supplier_id\" = supp.\"supplier_id\" order by item.\"supplier_id\"")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixClientProject(item_id=[$0], NAME=[$1], supplier_id=[$3], NAME0=[$4], supplier_id0=[$2])\n" +
 +                           "    PhoenixClientJoin(condition=[=($2, $3)], joinType=[inner])\n" +
 +                           "      PhoenixServerSort(sort0=[$2], dir0=[ASC])\n" +
 +                           "        PhoenixServerProject(item_id=[$0], NAME=[$1], supplier_id=[$5])\n" +
 +                           "          PhoenixTableScan(table=[[phoenix, Join, ItemTable]])\n" +
 +                           "      PhoenixServerProject(supplier_id=[$0], NAME=[$1])\n" +
 +                           "        PhoenixTableScan(table=[[phoenix, Join, SupplierTable]], scanOrder=[FORWARD])\n")
 +                .close();
 +        
 +        start(false).sql("SELECT \"order_id\", i.name, i.price, discount2, quantity FROM " + JOIN_ORDER_TABLE_FULL_NAME + " o LEFT JOIN " 
 +                + JOIN_ITEM_TABLE_FULL_NAME + " i ON o.\"item_id\" = i.\"item_id\" limit 2")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                		   "  PhoenixClientProject(order_id=[$0], NAME=[$4], PRICE=[$5], DISCOUNT2=[$6], QUANTITY=[$2])\n" +
 +                		   "    PhoenixLimit(fetch=[2])\n" +
 +                		   "      PhoenixClientJoin(condition=[=($1, $3)], joinType=[left])\n" +
 +                		   "        PhoenixClientSort(sort0=[$1], dir0=[ASC])\n" +
 +                		   "          PhoenixLimit(fetch=[2])\n" +
 +                		   "            PhoenixServerProject(order_id=[$0], item_id=[$2], QUANTITY=[$4])\n" +
 +                		   "              PhoenixTableScan(table=[[phoenix, Join, OrderTable]])\n" +
 +                		   "        PhoenixServerProject(item_id=[$0], NAME=[$1], PRICE=[$2], DISCOUNT2=[$4])\n" +
 +                		   "          PhoenixTableScan(table=[[phoenix, Join, ItemTable]], scanOrder=[FORWARD])\n")
 +                .close();
 +    }
 +    
 +    @Test public void testMultiJoin() throws Exception {
 +        start(false).sql("select t1.entity_id, t2.a_string, t3.organization_id from aTable t1 join aTable t2 on t1.entity_id = t2.entity_id and t1.organization_id = t2.organization_id join atable t3 on t1.entity_id = t3.entity_id and t1.organization_id = t3.organization_id where t1.a_string = 'a'") 
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixClientProject(ENTITY_ID=[$4], A_STRING=[$2], ORGANIZATION_ID=[$6])\n" +
 +                           "    PhoenixServerJoin(condition=[AND(=($4, $1), =($3, $0))], joinType=[inner])\n" +
 +                           "      PhoenixServerProject(ORGANIZATION_ID=[$0], ENTITY_ID=[$1], A_STRING=[$2])\n" +
 +                           "        PhoenixTableScan(table=[[phoenix, ATABLE]])\n" +
 +                           "      PhoenixClientProject(ORGANIZATION_ID=[$2], ENTITY_ID=[$3], A_STRING=[$4], ORGANIZATION_ID0=[$0], ENTITY_ID0=[$1])\n" +
 +                           "        PhoenixServerJoin(condition=[AND(=($3, $1), =($2, $0))], joinType=[inner])\n" +
 +                           "          PhoenixServerProject(ORGANIZATION_ID=[$0], ENTITY_ID=[$1])\n" +
 +                           "            PhoenixTableScan(table=[[phoenix, ATABLE]])\n" +
 +                           "          PhoenixServerProject(ORGANIZATION_ID=[$0], ENTITY_ID=[$1], A_STRING=[$2])\n" +
 +                           "            PhoenixTableScan(table=[[phoenix, ATABLE]], filter=[=($2, 'a')])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                          {"00A123122312312", "a", "00D300000000XHP"}, 
 +                          {"00A223122312312", "a", "00D300000000XHP"}, 
 +                          {"00A323122312312", "a", "00D300000000XHP"}, 
 +                          {"00A423122312312", "a", "00D300000000XHP"}})
 +                .close();
 +        
 +        start(false).sql("select t1.entity_id, t2.a_string, t3.organization_id from aTable t1 join aTable t2 on t1.entity_id = t2.entity_id and t1.organization_id = t2.organization_id join atable t3 on t1.entity_id = t3.entity_id and t1.organization_id = t3.organization_id")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixClientProject(ENTITY_ID=[$4], A_STRING=[$2], ORGANIZATION_ID=[$5])\n" +
 +                           "    PhoenixServerJoin(condition=[AND(=($4, $1), =($3, $0))], joinType=[inner])\n" +
 +                           "      PhoenixServerProject(ORGANIZATION_ID=[$0], ENTITY_ID=[$1], A_STRING=[$2])\n" +
 +                           "        PhoenixTableScan(table=[[phoenix, ATABLE]])\n" +
 +                           "      PhoenixServerJoin(condition=[AND(=($1, $3), =($0, $2))], joinType=[inner])\n" +
 +                           "        PhoenixServerProject(ORGANIZATION_ID=[$0], ENTITY_ID=[$1])\n" +
 +                           "          PhoenixTableScan(table=[[phoenix, ATABLE]])\n" +
 +                           "        PhoenixServerProject(ORGANIZATION_ID=[$0], ENTITY_ID=[$1])\n" +
 +                           "          PhoenixTableScan(table=[[phoenix, ATABLE]])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                          {"00A123122312312", "a", "00D300000000XHP"},
 +                          {"00A223122312312", "a", "00D300000000XHP"},
 +                          {"00A323122312312", "a", "00D300000000XHP"},
 +                          {"00A423122312312", "a", "00D300000000XHP"},
 +                          {"00B523122312312", "b", "00D300000000XHP"},
 +                          {"00B623122312312", "b", "00D300000000XHP"},
 +                          {"00B723122312312", "b", "00D300000000XHP"},
 +                          {"00B823122312312", "b", "00D300000000XHP"},
 +                          {"00C923122312312", "c", "00D300000000XHP"}})
 +                .close();
 +    }
 +    
 +    @Test public void testAggregate() throws Exception {
 +        start(false).sql("select count(b_string) from atable")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixServerAggregate(group=[{}], EXPR$0=[COUNT($3)])\n" +
 +                           "    PhoenixTableScan(table=[[phoenix, ATABLE]])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                          {9L}})
 +                .close();
 +        
 +        start(false).sql("select organization_id, count(b_string) from atable group by organization_id")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixServerAggregate(group=[{0}], EXPR$1=[COUNT($3)], isOrdered=[true])\n" +
 +                           "    PhoenixTableScan(table=[[phoenix, ATABLE]], scanOrder=[FORWARD])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                          {"00D300000000XHP", 9L}})
 +                .close();
 +        
 +        start(false).sql("select organization_id, entity_id, count(b_string) from atable group by entity_id ,organization_id")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixServerAggregate(group=[{0, 1}], EXPR$2=[COUNT($3)], isOrdered=[true])\n" +
 +                           "    PhoenixTableScan(table=[[phoenix, ATABLE]], scanOrder=[FORWARD])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                          {"00D300000000XHP", "00A123122312312", 1L}, 
 +                          {"00D300000000XHP", "00A223122312312", 1L}, 
 +                          {"00D300000000XHP", "00A323122312312", 1L}, 
 +                          {"00D300000000XHP", "00A423122312312", 1L}, 
 +                          {"00D300000000XHP", "00B523122312312", 1L}, 
 +                          {"00D300000000XHP", "00B623122312312", 1L}, 
 +                          {"00D300000000XHP", "00B723122312312", 1L}, 
 +                          {"00D300000000XHP", "00B823122312312", 1L}, 
 +                          {"00D300000000XHP", "00C923122312312", 1L}})
 +                .close();
 +        
 +        start(false).sql("select entity_id, count(b_string) from atable group by entity_id")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixServerAggregate(group=[{1}], EXPR$1=[COUNT($3)], isOrdered=[false])\n" +
 +                           "    PhoenixTableScan(table=[[phoenix, ATABLE]])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                        {"00A123122312312", 1L}, 
 +                        {"00A223122312312", 1L}, 
 +                        {"00A323122312312", 1L}, 
 +                        {"00A423122312312", 1L}, 
 +                        {"00B523122312312", 1L}, 
 +                        {"00B623122312312", 1L}, 
 +                        {"00B723122312312", 1L}, 
 +                        {"00B823122312312", 1L}, 
 +                        {"00C923122312312", 1L}})
 +                .close();
 +        
 +        start(false).sql("select a_string, count(b_string) from atable group by a_string")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixServerAggregate(group=[{2}], EXPR$1=[COUNT($3)], isOrdered=[false])\n" +
 +                           "    PhoenixTableScan(table=[[phoenix, ATABLE]])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                          {"a", 4L},
 +                          {"b", 4L},
 +                          {"c", 1L}})
 +                .close();
 +        
 +        start(false).sql("select count(entity_id), a_string from atable group by a_string")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixClientProject(EXPR$0=[$1], A_STRING=[$0])\n" +
 +                           "    PhoenixServerAggregate(group=[{2}], EXPR$0=[COUNT()], isOrdered=[false])\n" +
 +                           "      PhoenixTableScan(table=[[phoenix, ATABLE]])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                          {4L, "a"},
 +                          {4L, "b"},
 +                          {1L, "c"}})
 +                .close();
 +        
 +        start(false).sql("select s.name, count(\"item_id\") from " + JOIN_SUPPLIER_TABLE_FULL_NAME + " s join " + JOIN_ITEM_TABLE_FULL_NAME + " i on s.\"supplier_id\" = i.\"supplier_id\" group by s.name")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixServerAggregate(group=[{2}], EXPR$1=[COUNT()], isOrdered=[false])\n" +
 +                           "    PhoenixServerJoin(condition=[=($1, $0)], joinType=[inner])\n" +
 +                           "      PhoenixServerProject(supplier_id=[$5])\n" +
 +                           "        PhoenixTableScan(table=[[phoenix, Join, ItemTable]])\n" +
 +                           "      PhoenixServerProject(supplier_id=[$0], NAME=[$1])\n" +
 +                           "        PhoenixTableScan(table=[[phoenix, Join, SupplierTable]])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                          {"S1", 2L},
 +                          {"S2", 2L},
 +                          {"S5", 1L},
 +                          {"S6", 1L}})
 +                .close();
 +        
 +        // test PhoenixOrderedAggregateRule
 +        start(false).sql("select s.\"supplier_id\", count(*) from " + JOIN_SUPPLIER_TABLE_FULL_NAME + " s join " + JOIN_ITEM_TABLE_FULL_NAME + " i on s.\"supplier_id\" = i.\"supplier_id\" group by s.\"supplier_id\"")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixServerAggregate(group=[{0}], EXPR$1=[COUNT()], isOrdered=[true])\n" +
 +                           "    PhoenixServerJoin(condition=[=($0, $1)], joinType=[inner])\n" +
 +                           "      PhoenixServerProject(supplier_id=[$0])\n" +
 +                           "        PhoenixTableScan(table=[[phoenix, Join, SupplierTable]], scanOrder=[FORWARD])\n" +
 +                           "      PhoenixServerProject(supplier_id=[$5])\n" +
 +                           "        PhoenixTableScan(table=[[phoenix, Join, ItemTable]])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                          {"0000000001", 2L},
 +                          {"0000000002", 2L},
 +                          {"0000000005", 1L},
 +                          {"0000000006", 1L}})
 +                .close();
 +        
 +        start(false).sql("select a_string, sum(a_integer) from aTable group by a_string")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixServerAggregate(group=[{2}], EXPR$1=[SUM($4)], isOrdered=[false])\n" +
 +                           "    PhoenixTableScan(table=[[phoenix, ATABLE]])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                           {"a", 10L},
 +                           {"b", 26L},
 +                           {"c", 9L}})
 +                .close();
 +        
 +        start(false).sql("select mypk0, avg(mypk1) from " + SALTED_TABLE_NAME + " group by mypk0")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixClientProject(MYPK0=[$0], EXPR$1=[CAST(/($1, $2)):INTEGER NOT NULL])\n" +
 +                           "    PhoenixServerAggregate(group=[{0}], agg#0=[$SUM0($1)], agg#1=[COUNT()], isOrdered=[true])\n" +
 +                           "      PhoenixTableScan(table=[[phoenix, SALTED_TEST_TABLE]], scanOrder=[FORWARD])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                        {1, 2},
 +                        {2, 3},
 +                        {3, 4}})
 +                .close();
 +    }
 +    
 +    @Test public void testDistinct() throws Exception {
 +        start(false).sql("select distinct a_string from aTable")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixServerAggregate(group=[{2}], isOrdered=[false])\n" +
 +                           "    PhoenixTableScan(table=[[phoenix, ATABLE]])\n")
-                 .resultIs(new Object[][]{
++                .resultIs(false, new Object[][]{
 +                          {"a"}, 
 +                          {"b"}, 
 +                          {"c"}})
 +                .close();
 +    }
 +    
 +    @Test public void testSort() throws Exception {
 +        start(false).sql("select organization_id, entity_id, a_string from aTable order by a_string, entity_id")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixServerSort(sort0=[$2], sort1=[$1], dir0=[ASC], dir1=[ASC])\n" +
 +                           "    PhoenixServerProject(ORGANIZATION_ID=[$0], ENTITY_ID=[$1], A_STRING=[$2])\n" +
 +                           "      PhoenixTableScan(table=[[phoenix, ATABLE]])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(true, new Object[][] {
 +                          {"00D300000000XHP", "00A123122312312", "a"}, 
 +                          {"00D300000000XHP", "00A223122312312", "a"}, 
 +                          {"00D300000000XHP", "00A323122312312", "a"}, 
 +                          {"00D300000000XHP", "00A423122312312", "a"}, 
 +                          {"00D300000000XHP", "00B523122312312", "b"}, 
 +                          {"00D300000000XHP", "00B623122312312", "b"}, 
 +                          {"00D300000000XHP", "00B723122312312", "b"}, 
 +                          {"00D300000000XHP", "00B823122312312", "b"}, 
 +                          {"00D300000000XHP", "00C923122312312", "c"}})
 +                .close();
 +        
 +        start(false).sql("select organization_id, entity_id, a_string from aTable order by organization_id, entity_id")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixServerProject(ORGANIZATION_ID=[$0], ENTITY_ID=[$1], A_STRING=[$2])\n" +
 +                           "    PhoenixTableScan(table=[[phoenix, ATABLE]], scanOrder=[FORWARD])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(true, new Object[][] {
 +                          {"00D300000000XHP", "00A123122312312", "a"}, 
 +                          {"00D300000000XHP", "00A223122312312", "a"}, 
 +                          {"00D300000000XHP", "00A323122312312", "a"}, 
 +                          {"00D300000000XHP", "00A423122312312", "a"}, 
 +                          {"00D300000000XHP", "00B523122312312", "b"}, 
 +                          {"00D300000000XHP", "00B623122312312", "b"}, 
 +                          {"00D300000000XHP", "00B723122312312", "b"}, 
 +                          {"00D300000000XHP", "00B823122312312", "b"}, 
 +                          {"00D300000000XHP", "00C923122312312", "c"}})
 +                .close();
 +        
 +        start(false).sql("select organization_id, entity_id, a_string from aTable order by organization_id DESC")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixServerProject(ORGANIZATION_ID=[$0], ENTITY_ID=[$1], A_STRING=[$2])\n" +
 +                           "    PhoenixTableScan(table=[[phoenix, ATABLE]], scanOrder=[REVERSE])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(true, new Object[][] {
 +                          {"00D300000000XHP", "00C923122312312", "c"},
 +                          {"00D300000000XHP", "00B823122312312", "b"}, 
 +                          {"00D300000000XHP", "00B723122312312", "b"}, 
 +                          {"00D300000000XHP", "00B623122312312", "b"}, 
 +                          {"00D300000000XHP", "00B523122312312", "b"}, 
 +                          {"00D300000000XHP", "00A423122312312", "a"}, 
 +                          {"00D300000000XHP", "00A323122312312", "a"}, 
 +                          {"00D300000000XHP", "00A223122312312", "a"}, 
 +                          {"00D300000000XHP", "00A123122312312", "a"}})
 +                .close();
 +        
 +        start(false).sql("select organization_id, entity_id, a_string from aTable order by organization_id DESC, entity_id DESC")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixServerProject(ORGANIZATION_ID=[$0], ENTITY_ID=[$1], A_STRING=[$2])\n" +
 +                           "    PhoenixTableScan(table=[[phoenix, ATABLE]], scanOrder=[REVERSE])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(true, new Object[][] {
 +                          {"00D300000000XHP", "00C923122312312", "c"},
 +                          {"00D300000000XHP", "00B823122312312", "b"}, 
 +                          {"00D300000000XHP", "00B723122312312", "b"}, 
 +                          {"00D300000000XHP", "00B623122312312", "b"}, 
 +                          {"00D300000000XHP", "00B523122312312", "b"}, 
 +                          {"00D300000000XHP", "00A423122312312", "a"}, 
 +                          {"00D300000000XHP", "00A323122312312", "a"}, 
 +                          {"00D300000000XHP", "00A223122312312", "a"}, 
 +                          {"00D300000000XHP", "00A123122312312", "a"}})
 +                .close();
 +
 +        start(false).sql("select organization_id, entity_id, a_string from aTable order by organization_id ASC, entity_id DESC")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixServerSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[DESC])\n" +
 +                           "    PhoenixServerProject(ORGANIZATION_ID=[$0], ENTITY_ID=[$1], A_STRING=[$2])\n" +
 +                           "      PhoenixTableScan(table=[[phoenix, ATABLE]])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(true, new Object[][] {
 +                          {"00D300000000XHP", "00C923122312312", "c"},
 +                          {"00D300000000XHP", "00B823122312312", "b"}, 
 +                          {"00D300000000XHP", "00B723122312312", "b"}, 
 +                          {"00D300000000XHP", "00B623122312312", "b"}, 
 +                          {"00D300000000XHP", "00B523122312312", "b"}, 
 +                          {"00D300000000XHP", "00A423122312312", "a"}, 
 +                          {"00D300000000XHP", "00A323122312312", "a"}, 
 +                          {"00D300000000XHP", "00A223122312312", "a"}, 
 +                          {"00D300000000XHP", "00A123122312312", "a"}})
 +                .close();
 +        
 +        start(false).sql("select count(entity_id), a_string from atable group by a_string order by count(entity_id), a_string desc")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixClientProject(EXPR$0=[$1], A_STRING=[$0])\n" +
 +                           "    PhoenixCompactClientSort(sort0=[$1], sort1=[$0], dir0=[ASC], dir1=[DESC])\n" +
 +                           "      PhoenixServerAggregate(group=[{2}], EXPR$0=[COUNT()], isOrdered=[false])\n" +
 +                           "        PhoenixTableScan(table=[[phoenix, ATABLE]])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(true, new Object[][] {
 +                          {1L, "c"},
 +                          {4L, "b"},
 +                          {4L, "a"}})
 +                .close();
 +        
 +        start(false).sql("select s.name, count(\"item_id\") from " + JOIN_SUPPLIER_TABLE_FULL_NAME + " s join " + JOIN_ITEM_TABLE_FULL_NAME + " i on s.\"supplier_id\" = i.\"supplier_id\" group by s.name order by count(\"item_id\"), s.name desc")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixCompactClientSort(sort0=[$1], sort1=[$0], dir0=[ASC], dir1=[DESC])\n" +
 +                           "    PhoenixServerAggregate(group=[{2}], EXPR$1=[COUNT()], isOrdered=[false])\n" +
 +                           "      PhoenixServerJoin(condition=[=($1, $0)], joinType=[inner])\n" +
 +                           "        PhoenixServerProject(supplier_id=[$5])\n" +
 +                           "          PhoenixTableScan(table=[[phoenix, Join, ItemTable]])\n" +
 +                           "        PhoenixServerProject(supplier_id=[$0], NAME=[$1])\n" +
 +                           "          PhoenixTableScan(table=[[phoenix, Join, SupplierTable]])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(true, new Object[][] {
 +                          {"S6", 1L},
 +                          {"S5", 1L},
 +                          {"S2", 2L},
 +                          {"S1", 2L}})
 +                .close();
 +        
 +        start(false).sql("SELECT item.\"item_id\", item.name, supp.\"supplier_id\", supp.name FROM " + JOIN_ITEM_TABLE_FULL_NAME + " item JOIN " + JOIN_SUPPLIER_TABLE_FULL_NAME + " supp ON item.\"supplier_id\" = supp.\"supplier_id\" order by item.name desc")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixClientProject(item_id=[$0], NAME=[$1], supplier_id=[$3], NAME0=[$4])\n" +
 +                           "    PhoenixServerSort(sort0=[$1], dir0=[DESC])\n" +
 +                           "      PhoenixServerJoin(condition=[=($2, $3)], joinType=[inner])\n" +
 +                           "        PhoenixServerProject(item_id=[$0], NAME=[$1], supplier_id=[$5])\n" +
 +                           "          PhoenixTableScan(table=[[phoenix, Join, ItemTable]])\n" +
 +                           "        PhoenixServerProject(supplier_id=[$0], NAME=[$1])\n" +
 +                           "          PhoenixTableScan(table=[[phoenix, Join, SupplierTable]])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(true, new Object[][] {
 +                          {"0000000006", "T6", "0000000006", "S6"}, 
 +                          {"0000000005", "T5", "0000000005", "S5"}, 
 +                          {"0000000004", "T4", "0000000002", "S2"}, 
 +                          {"0000000003", "T3", "0000000002", "S2"},
 +                          {"0000000002", "T2", "0000000001", "S1"},
 +                          {"0000000001", "T1", "0000000001", "S1"}})
 +                .close();
 +    }
 +    
 +    @Test public void testSortWithLimit() throws Exception {
 +        start(false).sql("select organization_id, entity_id, a_string from aTable order by a_string, entity_id limit 5")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixLimit(fetch=[5])\n" +
 +                           "    PhoenixServerSort(sort0=[$2], sort1=[$1], dir0=[ASC], dir1=[ASC])\n" +
 +                           "      PhoenixServerProject(ORGANIZATION_ID=[$0], ENTITY_ID=[$1], A_STRING=[$2])\n" +
 +                           "        PhoenixTableScan(table=[[phoenix, ATABLE]])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(true, new Object[][] {
 +                          {"00D300000000XHP", "00A123122312312", "a"}, 
 +                          {"00D300000000XHP", "00A223122312312", "a"}, 
 +                          {"00D300000000XHP", "00A323122312312", "a"}, 
 +                          {"00D300000000XHP", "00A423122312312", "a"}, 
 +                          {"00D300000000XHP", "00B523122312312", "b"}})
 +                .close();
 +        
 +        start(false).sql("select organization_id, entity_id, a_string from aTable order by organization_id, entity_id limit 5")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixLimit(fetch=[5])\n" +
 +                           "    PhoenixServerProject(ORGANIZATION_ID=[$0], ENTITY_ID=[$1], A_STRING=[$2])\n" +
 +                           "      PhoenixTableScan(table=[[phoenix, ATABLE]], scanOrder=[FORWARD])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(true, new Object[][] {
 +                          {"00D300000000XHP", "00A123122312312", "a"}, 
 +                          {"00D300000000XHP", "00A223122312312", "a"}, 
 +                          {"00D300000000XHP", "00A323122312312", "a"}, 
 +                          {"00D300000000XHP", "00A423122312312", "a"}, 
 +                          {"00D300000000XHP", "00B523122312312", "b"}})
 +                .close();
 +        
 +        start(false).sql("select organization_id, entity_id, a_string from aTable order by organization_id DESC limit 5")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixLimit(fetch=[5])\n" +
 +                           "    PhoenixServerProject(ORGANIZATION_ID=[$0], ENTITY_ID=[$1], A_STRING=[$2])\n" +
 +                           "      PhoenixTableScan(table=[[phoenix, ATABLE]], scanOrder=[REVERSE])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(true, new Object[][] {
 +                          {"00D300000000XHP", "00C923122312312", "c"},
 +                          {"00D300000000XHP", "00B823122312312", "b"}, 
 +                          {"00D300000000XHP", "00B723122312312", "b"}, 
 +                          {"00D300000000XHP", "00B623122312312", "b"}, 
 +                          {"00D300000000XHP", "00B523122312312", "b"}})
 +                .close();
 +        
 +        start(false).sql("select organization_id, entity_id, a_string from aTable order by organization_id DESC, entity_id DESC limit 5")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixLimit(fetch=[5])\n" +
 +                           "    PhoenixServerProject(ORGANIZATION_ID=[$0], ENTITY_ID=[$1], A_STRING=[$2])\n" +
 +                           "      PhoenixTableScan(table=[[phoenix, ATABLE]], scanOrder=[REVERSE])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(true, new Object[][] {
 +                          {"00D300000000XHP", "00C923122312312", "c"},
 +                          {"00D300000000XHP", "00B823122312312", "b"}, 
 +                          {"00D300000000XHP", "00B723122312312", "b"}, 
 +                          {"00D300000000XHP", "00B623122312312", "b"}, 
 +                          {"00D300000000XHP", "00B523122312312", "b"}})
 +                .close();
 +
 +        start(false).sql("select organization_id, entity_id, a_string from aTable order by organization_id ASC, entity_id DESC limit 5")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixLimit(fetch=[5])\n" +
 +                           "    PhoenixServerSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[DESC])\n" +
 +                           "      PhoenixServerProject(ORGANIZATION_ID=[$0], ENTITY_ID=[$1], A_STRING=[$2])\n" +
 +                           "        PhoenixTableScan(table=[[phoenix, ATABLE]])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(true, new Object[][] {
 +                          {"00D300000000XHP", "00C923122312312", "c"},
 +                          {"00D300000000XHP", "00B823122312312", "b"}, 
 +                          {"00D300000000XHP", "00B723122312312", "b"}, 
 +                          {"00D300000000XHP", "00B623122312312", "b"}, 
 +                          {"00D300000000XHP", "00B523122312312", "b"}})
 +                .close();
 +        
 +        start(false).sql("select count(entity_id), a_string from atable group by a_string order by count(entity_id), a_string desc limit 2")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixClientProject(EXPR$0=[$1], A_STRING=[$0])\n" +
 +                           "    PhoenixLimit(fetch=[2])\n" +
 +                           "      PhoenixCompactClientSort(sort0=[$1], sort1=[$0], dir0=[ASC], dir1=[DESC])\n" +
 +                           "        PhoenixServerAggregate(group=[{2}], EXPR$0=[COUNT()], isOrdered=[false])\n" +
 +                           "          PhoenixTableScan(table=[[phoenix, ATABLE]])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(true, new Object[][] {
 +                          {1L, "c"},
 +                          {4L, "b"}})
 +                .close();
 +        
 +        start(false).sql("select s.name, count(\"item_id\") from " + JOIN_SUPPLIER_TABLE_FULL_NAME + " s join " + JOIN_ITEM_TABLE_FULL_NAME + " i on s.\"supplier_id\" = i.\"supplier_id\" group by s.name order by count(\"item_id\"), s.name desc limit 3")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixLimit(fetch=[3])\n" +
 +                           "    PhoenixCompactClientSort(sort0=[$1], sort1=[$0], dir0=[ASC], dir1=[DESC])\n" +
 +                           "      PhoenixServerAggregate(group=[{2}], EXPR$1=[COUNT()], isOrdered=[false])\n" +
 +                           "        PhoenixServerJoin(condition=[=($1, $0)], joinType=[inner])\n" +
 +                           "          PhoenixServerProject(supplier_id=[$5])\n" +
 +                           "            PhoenixTableScan(table=[[phoenix, Join, ItemTable]])\n" +
 +                           "          PhoenixServerProject(supplier_id=[$0], NAME=[$1])\n" +
 +                           "            PhoenixTableScan(table=[[phoenix, Join, SupplierTable]])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(true, new Object[][] {
 +                          {"S6", 1L},
 +                          {"S5", 1L},
 +                          {"S2", 2L}})
 +                .close();
 +        
 +        start(false).sql("SELECT item.\"item_id\", item.name, supp.\"supplier_id\", supp.name FROM " + JOIN_ITEM_TABLE_FULL_NAME + " item JOIN " + JOIN_SUPPLIER_TABLE_FULL_NAME + " supp ON item.\"supplier_id\" = supp.\"supplier_id\" order by item.name desc limit 3")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixClientProject(item_id=[$0], NAME=[$1], supplier_id=[$3], NAME0=[$4])\n" +
 +                           "    PhoenixLimit(fetch=[3])\n" +
 +                           "      PhoenixServerSort(sort0=[$1], dir0=[DESC])\n" +
 +                           "        PhoenixServerJoin(condition=[=($2, $3)], joinType=[inner])\n" +
 +                           "          PhoenixServerProject(item_id=[$0], NAME=[$1], supplier_id=[$5])\n" +
 +                           "            PhoenixTableScan(table=[[phoenix, Join, ItemTable]])\n" +
 +                           "          PhoenixServerProject(supplier_id=[$0], NAME=[$1])\n" +
 +                           "            PhoenixTableScan(table=[[phoenix, Join, SupplierTable]])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(true, new Object[][] {
 +                          {"0000000006", "T6", "0000000006", "S6"}, 
 +                          {"0000000005", "T5", "0000000005", "S5"}, 
 +                          {"0000000004", "T4", "0000000002", "S2"}})
 +                .close();
 +    }
 +    
 +    @Test public void testLimit() throws Exception {
 +        start(false).sql("select organization_id, entity_id, a_string from aTable limit 5")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixLimit(fetch=[5])\n" +
 +                           "    PhoenixServerProject(ORGANIZATION_ID=[$0], ENTITY_ID=[$1], A_STRING=[$2])\n" +
 +                           "      PhoenixTableScan(table=[[phoenix, ATABLE]])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                          {"00D300000000XHP", "00A123122312312", "a"}, 
 +                          {"00D300000000XHP", "00A223122312312", "a"}, 
 +                          {"00D300000000XHP", "00A323122312312", "a"}, 
 +                          {"00D300000000XHP", "00A423122312312", "a"}, 
 +                          {"00D300000000XHP", "00B523122312312", "b"}})
 +                .close();
 +        
 +        start(false).sql("select count(entity_id), a_string from atable group by a_string limit 2")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixClientProject(EXPR$0=[$1], A_STRING=[$0])\n" +
 +                           "    PhoenixLimit(fetch=[2])\n" +
 +                           "      PhoenixServerAggregate(group=[{2}], EXPR$0=[COUNT()], isOrdered=[false])\n" +
 +                           "        PhoenixTableScan(table=[[phoenix, ATABLE]])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                          {4L, "a"},
 +                          {4L, "b"}})
 +                .close();
 +        
 +        start(false).sql("select s.name, count(\"item_id\") from " + JOIN_SUPPLIER_TABLE_FULL_NAME + " s join " + JOIN_ITEM_TABLE_FULL_NAME + " i on s.\"supplier_id\" = i.\"supplier_id\" group by s.name limit 3")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixLimit(fetch=[3])\n" +
 +                           "    PhoenixServerAggregate(group=[{2}], EXPR$1=[COUNT()], isOrdered=[false])\n" +
 +                           "      PhoenixServerJoin(condition=[=($1, $0)], joinType=[inner])\n" +
 +                           "        PhoenixServerProject(supplier_id=[$5])\n" +
 +                           "          PhoenixTableScan(table=[[phoenix, Join, ItemTable]])\n" +
 +                           "        PhoenixServerProject(supplier_id=[$0], NAME=[$1])\n" +
 +                           "          PhoenixTableScan(table=[[phoenix, Join, SupplierTable]])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                          {"S1", 2L},
 +                          {"S2", 2L},
 +                          {"S5", 1L}})
 +                .close();
 +        
 +        start(false).sql("SELECT item.\"item_id\", item.name, supp.\"supplier_id\", supp.name FROM " + JOIN_ITEM_TABLE_FULL_NAME + " item JOIN " + JOIN_SUPPLIER_TABLE_FULL_NAME + " supp ON item.\"supplier_id\" = supp.\"supplier_id\" limit 3")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixClientProject(item_id=[$0], NAME=[$1], supplier_id=[$3], NAME0=[$4])\n" +
 +                           "    PhoenixLimit(fetch=[3])\n" +
 +                           "      PhoenixServerJoin(condition=[=($2, $3)], joinType=[inner])\n" +
 +                           "        PhoenixServerProject(item_id=[$0], NAME=[$1], supplier_id=[$5])\n" +
 +                           "          PhoenixTableScan(table=[[phoenix, Join, ItemTable]])\n" +
 +                           "        PhoenixServerProject(supplier_id=[$0], NAME=[$1])\n" +
 +                           "          PhoenixTableScan(table=[[phoenix, Join, SupplierTable]])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                          {"0000000001", "T1", "0000000001", "S1"}, 
 +                          {"0000000002", "T2", "0000000001", "S1"}, 
 +                          {"0000000003", "T3", "0000000002", "S2"}})
 +                .close();
 +        
 +        start(false).sql("SELECT x from (values (1, 2), (2, 4), (3, 6)) as t(x, y) limit 2")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixClientProject(X=[$0])\n" +
 +                           "    PhoenixLimit(fetch=[2])\n" +
 +                           "      PhoenixValues(tuples=[[{ 1, 2 }, { 2, 4 }, { 3, 6 }]])\n")
-                 .resultIs(new Object[][] {{1}, {2}})
++                .resultIs(false, new Object[][] {{1}, {2}})
 +                .close();
 +    }
 +
 +    @Ignore // CALCITE-1045
 +    @Test public void testScalarSubquery() throws Exception {
 +        start(false).sql("select \"item_id\", name, (select max(quantity) sq \n"
 +            + "from " + JOIN_ORDER_TABLE_FULL_NAME + " o where o.\"item_id\" = i.\"item_id\")\n"
 +            + "from " + JOIN_ITEM_TABLE_FULL_NAME + " i")
 +            .explainIs("PhoenixToEnumerableConverter\n" +
 +                       "  PhoenixClientProject(item_id=[$0], NAME=[$1], EXPR$2=[$8])\n" +
 +                       "    PhoenixServerJoin(condition=[=($0, $7)], joinType=[left], isSingleValueRhs=[true])\n" +
 +                       "      PhoenixTableScan(table=[[phoenix, Join, ItemTable]])\n" +
 +                       "      PhoenixServerAggregate(group=[{0}], SQ=[MAX($5)], isOrdered=[true])\n" +
 +                       "        PhoenixServerJoin(condition=[=($3, $0)], joinType=[inner])\n" +
 +                       "          PhoenixServerProject(item_id=[$0])\n" +
 +                       "            PhoenixTableScan(table=[[phoenix, Join, ItemTable]], scanOrder=[FORWARD])\n" +
 +                       "          PhoenixTableScan(table=[[phoenix, Join, OrderTable]])\n")
-             .resultIs(new Object[][] {
++            .resultIs(false, new Object[][] {
 +                    new Object[] {"0000000001", "T1", 1000},
 +                    new Object[] {"0000000002", "T2", 3000},
 +                    new Object[] {"0000000003", "T3", 5000},
 +                    new Object[] {"0000000004", "T4", null},
 +                    new Object[] {"0000000005", "T5", null},
 +                    new Object[] {"0000000006", "T6", 4000},
 +                    new Object[] {"invalid001", "INVALID-1", null}})
 +            .close();
 +        
 +        start(false).sql("select \"item_id\", name, (select quantity sq \n"
 +                    + "from " + JOIN_ORDER_TABLE_FULL_NAME + " o where o.\"item_id\" = i.\"item_id\")\n"
 +                    + "from " + JOIN_ITEM_TABLE_FULL_NAME + " i where \"item_id\" < '0000000006'")
 +               .explainIs("PhoenixToEnumerableConverter\n" +
 +                          "  PhoenixClientProject(item_id=[$0], NAME=[$1], EXPR$2=[$8])\n" +
 +                          "    PhoenixServerJoin(condition=[=($0, $7)], joinType=[left], isSingleValueRhs=[true])\n" +
 +                          "      PhoenixTableScan(table=[[phoenix, Join, ItemTable]], filter=[<($0, '0000000006')])\n" +
 +                          "      PhoenixClientProject(item_id0=[$7], SQ=[$4])\n" +
 +                          "        PhoenixServerJoin(condition=[=($2, $7)], joinType=[inner])\n" +
 +                          "          PhoenixTableScan(table=[[phoenix, Join, OrderTable]])\n" +
 +                          "          PhoenixServerProject(item_id=[$0])\n" +
 +                          "            PhoenixTableScan(table=[[phoenix, Join, ItemTable]], filter=[<($0, '0000000006')])\n")
-                .resultIs(new Object[][] {
++               .resultIs(false, new Object[][] {
 +                         new Object[] {"0000000001", "T1", 1000},
 +                         new Object[] {"0000000002", "T2", 3000},
 +                         new Object[] {"0000000003", "T3", 5000},
 +                         new Object[] {"0000000004", "T4", null},
 +                         new Object[] {"0000000005", "T5", null}})
 +               .close();;
 +    }
 +    
 +    @Test public void testValues() throws Exception {
 +        start(false).sql("select p0+p1 from (values (2, 1)) as t(p0, p1)")
 +            .explainIs("PhoenixToEnumerableConverter\n" +
 +                       "  PhoenixClientProject(EXPR$0=[+($0, $1)])\n" +
 +                       "    PhoenixValues(tuples=[[{ 2, 1 }]])\n")
 +            .close();
 +        start(false).sql("select count(p0), max(p1) from (values (2, 1), (3, 4), (5, 2)) as t(p0, p1)")
 +            .explainIs("PhoenixToEnumerableConverter\n" +
 +                       "  PhoenixClientAggregate(group=[{}], EXPR$0=[COUNT()], EXPR$1=[MAX($1)])\n" +
 +                       "    PhoenixValues(tuples=[[{ 2, 1 }, { 3, 4 }, { 5, 2 }]])\n")
-             .resultIs(new Object[][] {{3L, 4}})
++            .resultIs(false, new Object[][] {{3L, 4}})
 +            .close();
 +    }
 +    
 +    @Test public void testUnion() throws Exception {
 +        start(false).sql("select entity_id from atable where a_string = 'a' union all select entity_id from atable where a_string = 'b'")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixUnion(all=[true])\n" +
 +                           "    PhoenixServerProject(ENTITY_ID=[$1])\n" +
 +                           "      PhoenixTableScan(table=[[phoenix, ATABLE]], filter=[=($2, 'a')])\n" +
 +                           "    PhoenixServerProject(ENTITY_ID=[$1])\n" +
 +                           "      PhoenixTableScan(table=[[phoenix, ATABLE]], filter=[=($2, 'b')])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                        {"00A123122312312"},
 +                        {"00A223122312312"},
 +                        {"00A323122312312"},
 +                        {"00A423122312312"},
 +                        {"00B523122312312"},
 +                        {"00B623122312312"},
 +                        {"00B723122312312"},
 +                        {"00B823122312312"}})
 +                .close();
 +        
 +        start(false).sql("select entity_id, a_string from atable where a_string = 'a' union all select entity_id, a_string from atable where a_string = 'c' order by entity_id desc limit 3")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixLimit(fetch=[3])\n" +
 +                           "    PhoenixMergeSortUnion(all=[true])\n" +
 +                           "      PhoenixLimit(fetch=[3])\n" +
 +                           "        PhoenixServerSort(sort0=[$0], dir0=[DESC])\n" +
 +                           "          PhoenixServerProject(ENTITY_ID=[$1], A_STRING=[$2])\n" +
 +                           "            PhoenixTableScan(table=[[phoenix, ATABLE]], filter=[=($2, 'a')])\n" +
 +                           "      PhoenixLimit(fetch=[3])\n" +
 +                           "        PhoenixServerSort(sort0=[$0], dir0=[DESC])\n" +
 +                           "          PhoenixServerProject(ENTITY_ID=[$1], A_STRING=[$2])\n" +
 +                           "            PhoenixTableScan(table=[[phoenix, ATABLE]], filter=[=($2, 'c')])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(true, new Object[][] {
 +                        {"00C923122312312", "c"},
 +                        {"00A423122312312", "a"},
 +                        {"00A323122312312", "a"}})
 +                .close();
 +        
 +        start(false).sql("select entity_id, a_string from atable where a_string = 'a' union all select entity_id, a_string from atable where a_string = 'c' order by entity_id desc")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixMergeSortUnion(all=[true])\n" +
 +                           "    PhoenixServerSort(sort0=[$0], dir0=[DESC])\n" +
 +                           "      PhoenixServerProject(ENTITY_ID=[$1], A_STRING=[$2])\n" +
 +                           "        PhoenixTableScan(table=[[phoenix, ATABLE]], filter=[=($2, 'a')])\n" +
 +                           "    PhoenixServerSort(sort0=[$0], dir0=[DESC])\n" +
 +                           "      PhoenixServerProject(ENTITY_ID=[$1], A_STRING=[$2])\n" +
 +                           "        PhoenixTableScan(table=[[phoenix, ATABLE]], filter=[=($2, 'c')])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(true, new Object[][] {
 +                        {"00C923122312312", "c"},
 +                        {"00A423122312312", "a"},
 +                        {"00A323122312312", "a"},
 +                        {"00A223122312312", "a"},
 +                        {"00A123122312312", "a"}})
 +                .close();
 +    }
 +    
 +    @Test public void testUnnest() throws Exception {
 +        start(false).sql("SELECT t.s FROM UNNEST((SELECT scores FROM " + SCORES_TABLE_NAME + ")) AS t(s)")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixUncollect\n" +
 +                           "    PhoenixServerProject(EXPR$0=[$2])\n" +
 +                           "      PhoenixTableScan(table=[[phoenix, SCORES]])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                        {85}, 
 +                        {80}, 
 +                        {82}, 
 +                        {87}, 
 +                        {88}, 
 +                        {80}})
 +                .close();
 +        start(false).sql("SELECT s.student_id, t.score FROM " + SCORES_TABLE_NAME + " s, UNNEST((SELECT scores FROM " + SCORES_TABLE_NAME + " s2 where s.student_id = s2.student_id)) AS t(score)")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixClientProject(STUDENT_ID=[$0], SCORE=[$3])\n" +
 +                           "    PhoenixCorrelate(correlation=[$cor0], joinType=[INNER], requiredColumns=[{0}])\n" +
 +                           "      PhoenixTableScan(table=[[phoenix, SCORES]])\n" +
 +                           "      PhoenixUncollect\n" +
 +                           "        PhoenixServerProject(EXPR$0=[$2])\n" +
 +                           "          PhoenixTableScan(table=[[phoenix, SCORES]], filter=[=($cor0.STUDENT_ID, $0)])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                        {1, 85}, 
 +                        {1, 80}, 
 +                        {1, 82}, 
 +                        {3, 87}, 
 +                        {3, 88}, 
 +                        {3, 80}})
 +                .close();
 +    }
 +    
 +    @Test public void testCorrelateAndDecorrelation() throws Exception {
 +        Properties correlProps = getConnectionProps(false);
 +        correlProps.setProperty("forceDecorrelate", Boolean.FALSE.toString());
 +        Properties decorrelProps = getConnectionProps(false);
 +        decorrelProps.setProperty("forceDecorrelate", Boolean.TRUE.toString());
 +        
 +        String q1 = "select \"order_id\", quantity from " + JOIN_ORDER_TABLE_FULL_NAME + " o where quantity = (select max(quantity) from " + JOIN_ORDER_TABLE_FULL_NAME + " o2 where o.\"item_id\" = o2.\"item_id\")";
 +        Object[][] r1 = new Object[][] {
 +                {"000000000000001", 1000},
 +                {"000000000000003", 3000},
 +                {"000000000000004", 4000},
 +                {"000000000000005", 5000}};
 +        String p1Correlate = 
 +                "PhoenixToEnumerableConverter\n" +
 +                "  PhoenixClientProject(order_id=[$0], QUANTITY=[$2])\n" +
 +                "    PhoenixFilter(condition=[=($2, $3)])\n" +
 +                "      PhoenixCorrelate(correlation=[$cor0], joinType=[LEFT], requiredColumns=[{1}])\n" +
 +                "        PhoenixServerProject(order_id=[$0], item_id=[$2], QUANTITY=[$4])\n" +
 +                "          PhoenixTableScan(table=[[phoenix, Join, OrderTable]])\n" +
 +                "        PhoenixServerAggregate(group=[{}], EXPR$0=[MAX($4)])\n" +
 +                "          PhoenixTableScan(table=[[phoenix, Join, OrderTable]], filter=[=($cor0.item_id, $2)])\n";
 +        String p1Decorrelated = 
 +                "PhoenixToEnumerableConverter\n" +
 +                "  PhoenixClientProject(order_id=[$0], QUANTITY=[$2])\n" +
 +                "    PhoenixServerJoin(condition=[AND(=($1, $3), =($2, $4))], joinType=[inner])\n" +
 +                "      PhoenixServerProject(order_id=[$0], item_id=[$2], QUANTITY=[$4])\n" +
 +                "        PhoenixTableScan(table=[[phoenix, Join, OrderTable]])\n" +
 +                "      PhoenixServerAggregate(group=[{2}], EXPR$0=[MAX($1)], isOrdered=[false])\n" +
 +                "        PhoenixServerJoin(condition=[=($2, $0)], joinType=[inner])\n" +
 +                "          PhoenixServerProject(item_id=[$2], QUANTITY=[$4])\n" +
 +                "            PhoenixTableScan(table=[[phoenix, Join, OrderTable]])\n" +
 +                "          PhoenixServerAggregate(group=[{2}], isOrdered=[false])\n" +
 +                "            PhoenixTableScan(table=[[phoenix, Join, OrderTable]])\n";
-         start(correlProps).sql(q1).explainIs(p1Correlate).resultIs(r1).close();
-         start(decorrelProps).sql(q1).explainIs(p1Decorrelated).resultIs(r1).close();
++        start(correlProps).sql(q1).explainIs(p1Correlate).resultIs(false, r1).close();
++        start(decorrelProps).sql(q1).explainIs(p1Decorrelated).resultIs(false, r1).close();
 +                
 +        String q2 = "select name from " + JOIN_ITEM_TABLE_FULL_NAME + " i where price = (select max(price) from " + JOIN_ITEM_TABLE_FULL_NAME + " i2 where i.\"item_id\" = i2.\"item_id\" and i.name = i2.name and i2.\"item_id\" <> 'invalid001')";
 +        Object[][] r2 = new Object[][]{
 +                {"T1"},
 +                {"T2"},
 +                {"T3"},
 +                {"T4"},
 +                {"T5"},
 +                {"T6"}};
 +        String p2Correlate = 
 +                "PhoenixToEnumerableConverter\n" +
 +                "  PhoenixClientProject(NAME=[$1])\n" +
 +                "    PhoenixFilter(condition=[=($2, $3)])\n" +
 +                "      PhoenixCorrelate(correlation=[$cor0], joinType=[LEFT], requiredColumns=[{0, 1}])\n" +
 +                "        PhoenixServerProject(item_id=[$0], NAME=[$1], PRICE=[$2])\n" +
 +                "          PhoenixTableScan(table=[[phoenix, Join, ItemTable]])\n" +
 +                "        PhoenixServerAggregate(group=[{}], EXPR$0=[MAX($2)])\n" +
 +                "          PhoenixTableScan(table=[[phoenix, Join, ItemTable]], filter=[AND(=($cor0.item_id, $0), =($cor0.NAME, $1), <>($0, 'invalid001'))])\n";
 +        String p2Decorrelated = 
 +                "PhoenixToEnumerableConverter\n" +
 +                "  PhoenixClientProject(NAME=[$1])\n" +
 +                "    PhoenixServerJoin(condition=[AND(=($0, $3), =($1, $4), =($2, $5))], joinType=[inner])\n" +
 +                "      PhoenixServerProject(item_id=[$0], NAME=[$1], PRICE=[$2])\n" +
 +                "        PhoenixTableScan(table=[[phoenix, Join, ItemTable]])\n" +
 +                "      PhoenixServerAggregate(group=[{0, 1}], EXPR$0=[MAX($4)], isOrdered=[false])\n" +
 +                "        PhoenixServerJoin(condition=[AND(=($0, $2), =($1, $3))], joinType=[inner])\n" +
 +                "          PhoenixServerProject(item_id=[$0], NAME=[$1])\n" +
 +                "            PhoenixTableScan(table=[[phoenix, Join, ItemTable]])\n" +
 +                "          PhoenixServerProject(item_id=[$0], NAME=[$1], PRICE=[$2])\n" +
 +                "            PhoenixTableScan(table=[[phoenix, Join, ItemTable]], filter=[<>($0, 'invalid001')])\n";
-         start(correlProps).sql(q2).explainIs(p2Correlate).resultIs(r2).close();
-         start(decorrelProps).sql(q2).explainIs(p2Decorrelated).resultIs(r2).close();
++        start(correlProps).sql(q2).explainIs(p2Correlate).resultIs(false, r2).close();
++        start(decorrelProps).sql(q2).explainIs(p2Decorrelated).resultIs(false, r2).close();
 +        
 +        // Test PhoenixClientSemiJoin
 +        String q3 = "select \"item_id\", name from " + JOIN_ITEM_TABLE_FULL_NAME + " i where exists (select 1 from " + JOIN_ORDER_TABLE_FULL_NAME + " o where i.\"item_id\" = o.\"item_id\")";
 +        Object[][] r3 = new Object[][] {
 +                {"0000000001", "T1"},
 +                {"0000000002", "T2"},
 +                {"0000000003", "T3"},
 +                {"0000000006", "T6"}};
 +        String p3Correlate = 
 +                "PhoenixToEnumerableConverter\n" +
 +                "  PhoenixClientProject(item_id=[$0], NAME=[$1])\n" +
 +                "    PhoenixCorrelate(correlation=[$cor0], joinType=[INNER], requiredColumns=[{0}])\n" +
 +                "      PhoenixServerProject(item_id=[$0], NAME=[$1])\n" +
 +                "        PhoenixTableScan(table=[[phoenix, Join, ItemTable]])\n" +
 +                "      PhoenixServerAggregate(group=[{0}], isOrdered=[false])\n" +
 +                "        PhoenixServerProject(i=[true])\n" +
 +                "          PhoenixTableScan(table=[[phoenix, Join, OrderTable]], filter=[=($cor0.item_id, $2)])\n";
 +        String p3Decorrelated = 
 +                "PhoenixToEnumerableConverter\n" +
 +                "  PhoenixClientSemiJoin(condition=[=($0, $2)], joinType=[inner])\n" +
 +                "    PhoenixServerProject(item_id=[$0], NAME=[$1])\n" +
 +                "      PhoenixTableScan(table=[[phoenix, Join, ItemTable]], scanOrder=[FORWARD])\n" +
 +                "    PhoenixServerJoin(condition=[=($0, $1)], joinType=[inner])\n" +
 +                "      PhoenixServerProject(item_id=[$0])\n" +
 +                "        PhoenixTableScan(table=[[phoenix, Join, ItemTable]], scanOrder=[FORWARD])\n" +
 +                "      PhoenixServerProject(item_id=[$2])\n" +
 +                "        PhoenixTableScan(table=[[phoenix, Join, OrderTable]])\n";
-         start(correlProps).sql(q3).explainIs(p3Correlate).resultIs(r3).close();
-         start(decorrelProps).sql(q3).explainIs(p3Decorrelated).resultIs(r3).close();
++        start(correlProps).sql(q3).explainIs(p3Correlate).resultIs(false, r3).close();
++        start(decorrelProps).sql(q3).explainIs(p3Decorrelated).resultIs(false, r3).close();
 +        
 +        String q4 = "select \"item_id\", name from " + JOIN_ITEM_TABLE_FULL_NAME + " i where \"item_id\" in (select \"item_id\" from " + JOIN_ORDER_TABLE_FULL_NAME + ")";
 +        Object[][] r4 = new Object[][] {
 +                {"0000000001", "T1"},
 +                {"0000000002", "T2"},
 +                {"0000000003", "T3"},
 +                {"0000000006", "T6"}};
 +        String p4Decorrelated = 
 +                "PhoenixToEnumerableConverter\n" +
 +                "  PhoenixServerSemiJoin(condition=[=($0, $4)], joinType=[inner])\n" +
 +                "    PhoenixServerProject(item_id=[$0], NAME=[$1])\n" +
 +                "      PhoenixTableScan(table=[[phoenix, Join, ItemTable]])\n" +
 +                "    PhoenixTableScan(table=[[phoenix, Join, OrderTable]])\n";
-         start(decorrelProps).sql(q4).explainIs(p4Decorrelated).resultIs(r4).close();
++        start(decorrelProps).sql(q4).explainIs(p4Decorrelated).resultIs(false, r4).close();
 +        
 +        // CALCITE-864
 +        //String q5 = "select \"order_id\" from " + JOIN_ITEM_TABLE_FULL_NAME + " i JOIN " + JOIN_ORDER_TABLE_FULL_NAME + " o on o.\"item_id\" = i.\"item_id\" where quantity = (select max(quantity) from " + JOIN_ORDER_TABLE_FULL_NAME + " o2 JOIN " + JOIN_ITEM_TABLE_FULL_NAME + " i2 on o2.\"item_id\" = i2.\"item_id\" where i.\"supplier_id\" = i2.\"supplier_id\")";
 +        //Object [][] r5 = new Object[][] {
 +        //        {"000000000000003"},
 +        //        {"000000000000005"},
 +        //        {"000000000000004"}};
 +        //String p5Correlate = 
 +        //        "PhoenixToEnumerableConverter\n" +
 +        //        "  PhoenixClientProject(order_id=[$7])\n" +
 +        //        "    PhoenixFilter(condition=[=($11, $14)])\n" +
 +        //        "      PhoenixCorrelate(correlation=[$cor0], joinType=[LEFT], requiredColumns=[{5}])\n" +
 +        //        "        PhoenixServerJoin(condition=[=($9, $0)], joinType=[inner])\n" +
 +        //        "          PhoenixTableScan(table=[[phoenix, Join, ItemTable]])\n" +
 +        //        "          PhoenixTableScan(table=[[phoenix, Join, OrderTable]])\n" +
 +        //        "        PhoenixServerAggregate(group=[{}], EXPR$0=[MAX($4)])\n" +
 +        //        "          PhoenixServerJoin(condition=[=($2, $7)], joinType=[inner])\n" +
 +        //        "            PhoenixTableScan(table=[[phoenix, Join, OrderTable]])\n" +
 +        //        "            PhoenixTableScan(table=[[phoenix, Join, ItemTable]], filter=[=($cor0.supplier_id, $5)])\n";
 +        //String p5Decorrelated =
 +        //        "PhoenixToEnumerableConverter\n" +
 +        //        "  PhoenixClientProject(order_id=[$7])\n" +
 +        //        "    PhoenixServerJoin(condition=[AND(=($9, $0), =($5, $14))], joinType=[inner])\n" +
 +        //        "      PhoenixTableScan(table=[[phoenix, Join, ItemTable]])\n" +
 +        //        "      PhoenixServerJoin(condition=[=($4, $8)], joinType=[inner])\n" +
 +        //        "        PhoenixTableScan(table=[[phoenix, Join, OrderTable]])\n" +
 +        //        "        PhoenixServerAggregate(group=[{14}], EXPR$0=[MAX($4)], isOrdered=[false])\n" +
 +        //        "          PhoenixServerJoin(condition=[=($2, $7)], joinType=[inner])\n" +
 +        //        "            PhoenixTableScan(table=[[phoenix, Join, OrderTable]])\n" +
 +        //        "            PhoenixServerJoin(condition=[=($7, $5)], joinType=[inner])\n" +
 +        //        "              PhoenixTableScan(table=[[phoenix, Join, ItemTable]])\n" +
 +        //        "              PhoenixServerAggregate(group=[{5}], isOrdered=[false])\n" +
 +        //        "                PhoenixServerJoin(condition=[=($9, $0)], joinType=[inner])\n" +
 +        //        "                  PhoenixTableScan(table=[[phoenix, Join, ItemTable]])\n" +
 +        //        "                  PhoenixTableScan(table=[[phoenix, Join, OrderTable]])\n";
-         //start(correlProps).sql(q5).explainIs(p5Correlate).resultIs(r5).close();
-         //start(decorrelProps).sql(q5).explainIs(p5Decorrelated).resultIs(r5).close();
++        //start(correlProps).sql(q5).explainIs(p5Correlate).resultIs(false, r5).close();
++        //start(decorrelProps).sql(q5).explainIs(p5Decorrelated).resultIs(false, r5).close();
 +        
 +        String q6 = "select organization_id, entity_id, a_integer from v v1 where a_integer = (select min(a_integer) from v v2 where v1.organization_id = v2.organization_id and v1.b_string = v2.b_string)";
 +        Object[][] r6 = new Object[][] {
 +                {"00D300000000XHP", "00A123122312312", 1}, 
 +                {"00D300000000XHP", "00A223122312312", 2}, 
 +                {"00D300000000XHP", "00A323122312312", 3}};
 +        String p6Correlate = 
 +                "PhoenixToEnumerableConverter\n" +
 +                "  PhoenixClientProject(ORGANIZATION_ID=[$0], ENTITY_ID=[$1], A_INTEGER=[$3])\n" +
 +                "    PhoenixFilter(condition=[=($3, $4)])\n" +
 +                "      PhoenixCorrelate(correlation=[$cor0], joinType=[LEFT], requiredColumns=[{0, 2}])\n" +
 +                "        PhoenixServerProject(ORGANIZATION_ID=[$0], ENTITY_ID=[$1], B_STRING=[$3], A_INTEGER=[$4])\n" +
 +                "          PhoenixTableScan(table=[[phoenix, ATABLE]], filter=[=($2, 'a')])\n" +
 +                "        PhoenixServerAggregate(group=[{}], EXPR$0=[MIN($4)])\n" +
 +                "          PhoenixTableScan(table=[[phoenix, ATABLE]], filter=[AND(=($2, 'a'), =($cor0.ORGANIZATION_ID, $0), =($cor0.B_STRING, $3))])\n";
 +        String p6Decorrelated = 
 +                "PhoenixToEnumerableConverter\n" +
 +                "  PhoenixClientProject(ORGANIZATION_ID=[$0], ENTITY_ID=[$1], A_INTEGER=[$3])\n" +
 +                "    PhoenixServerJoin(condition=[AND(=($0, $4), =($2, $5), =($3, $6))], joinType=[inner])\n" +
 +                "      PhoenixServerProject(ORGANIZATION_ID=[$0], ENTITY_ID=[$1], B_STRING=[$3], A_INTEGER=[$4])\n" +
 +                "        PhoenixTableScan(table=[[phoenix, ATABLE]], filter=[=($2, 'a')])\n" +
 +                "      PhoenixServerAggregate(group=[{3, 4}], EXPR$0=[MIN($2)], isOrdered=[false])\n" +
 +                "        PhoenixServerJoin(condition=[AND(=($3, $0), =($4, $1))], joinType=[inner])\n" +
 +                "          PhoenixServerProject(ORGANIZATION_ID=[$0], B_STRING=[$3], A_INTEGER=[$4])\n" +
 +                "            PhoenixTableScan(table=[[phoenix, ATABLE]], filter=[=($2, 'a')])\n" +
 +                "          PhoenixServerAggregate(group=[{0, 3}], isOrdered=[false])\n" +
 +                "            PhoenixTableScan(table=[[phoenix, ATABLE]], filter=[=($2, 'a')])\n";
-         start(correlProps).sql(q6).explainIs(p6Correlate).resultIs(r6).close();
-         start(decorrelProps).sql(q6).explainIs(p6Decorrelated).resultIs(r6).close();
++        start(correlProps).sql(q6).explainIs(p6Correlate).resultIs(false, r6).close();
++        start(decorrelProps).sql(q6).explainIs(p6Decorrelated).resultIs(false, r6).close();
 +    }
 +    
 +    @Test public void testInValueList() throws Exception {
 +        start(false).sql("select entity_id from aTable where organization_id = '00D300000000XHP' and entity_id in ('00A123122312312', '00A223122312312', '00B523122312312', '00B623122312312', '00C923122312312')")
 +            .explainIs("PhoenixToEnumerableConverter\n" +
 +                       "  PhoenixServerProject(ENTITY_ID=[$1])\n" +
 +                       "    PhoenixTableScan(table=[[phoenix, ATABLE]], filter=[AND(=($0, '00D300000000XHP'), OR(=($1, '00A123122312312'), =($1, '00A223122312312'), =($1, '00B523122312312'), =($1, '00B623122312312'), =($1, '00C923122312312')))])\n")
-             .resultIs(new Object[][] {
++            .resultIs(false, new Object[][] {
 +                    {"00A123122312312"},
 +                    {"00A223122312312"},
 +                    {"00B523122312312"},
 +                    {"00B623122312312"},
 +                    {"00C923122312312"}})
 +            .close();
 +    }
 +    
 +    @Test public void testSelectFromView() throws Exception {
 +        start(false).sql("select * from v")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixTableScan(table=[[phoenix, ATABLE]], filter=[=($2, 'a')])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                        {"00D300000000XHP", "00A123122312312", "a"}, 
 +                        {"00D300000000XHP", "00A223122312312", "a"}, 
 +                        {"00D300000000XHP", "00A323122312312", "a"}, 
 +                        {"00D300000000XHP", "00A423122312312", "a"}})
 +                .close();
 +    }
 +    
 +    @Test public void testSequence() throws Exception {
 +        start(false).sql("select NEXT VALUE FOR seq0, c0 from (values (1), (1)) as t(c0)")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixClientProject(EXPR$0=[NEXT_VALUE('\"SEQ0\"')], C0=[$0])\n" +
 +                           "    PhoenixValues(tuples=[[{ 1 }, { 1 }]])\n")
-                 .resultIs(new Object[][]{
++                .resultIs(false, new Object[][]{
 +                        {1L, 1},
 +                        {2L, 1}})
 +                .close();
 +
 +        start(false).sql("select NEXT VALUE FOR my.seq1, entity_id from aTable where a_string = 'a'")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixClientProject(EXPR$0=[NEXT_VALUE('\"MY\".\"SEQ1\"')], ENTITY_ID=[$1])\n" +
 +                           "    PhoenixTableScan(table=[[phoenix, ATABLE]], filter=[=($2, 'a')])\n")
-                 .resultIs(new Object[][]{
++                .resultIs(false, new Object[][]{
 +                        {2L, "00A123122312312"},
 +                        {4L, "00A223122312312"},
 +                        {6L, "00A323122312312"},
 +                        {8L, "00A423122312312"}})
 +                .close();
 +        
 +        start(false).sql("SELECT NEXT VALUE FOR seq0, item.\"item_id\", item.name, supp.\"supplier_id\", supp.name FROM " + JOIN_ITEM_TABLE_FULL_NAME + " item JOIN " + JOIN_SUPPLIER_TABLE_FULL_NAME + " supp ON item.\"supplier_id\" = supp.\"supplier_id\"")
 +                .explainIs("PhoenixToEnumerableConverter\n" +
 +                           "  PhoenixClientProject(EXPR$0=[NEXT_VALUE('\"SEQ0\"')], item_id=[$0], NAME=[$1], supplier_id=[$3], NAME0=[$4])\n" +
 +                           "    PhoenixServerJoin(condition=[=($2, $3)], joinType=[inner])\n" +
 +                           "      PhoenixServerProject(item_id=[$0], NAME=[$1], supplier_id=[$5])\n" +
 +                           "        PhoenixTableScan(table=[[phoenix, Join, ItemTable]])\n" +
 +                           "      PhoenixServerProject(supplier_id=[$0], NAME=[$1])\n" +
 +                           "        PhoenixTableScan(table=[[phoenix, Join, SupplierTable]])\n")
-                 .resultIs(new Object[][] {
++                .resultIs(false, new Object[][] {
 +                        {3L, "0000000001", "T1", "0000000001", "S1"}, 
 +                        {4L, "0000000002", "T2", "0000000001", "S1"}, 
 +                        {5L, "0000000003", "T3", "0000000002", "S2"}, 
 +                        {6L, "0000000004", "T4", "0000000002", "S2"},
 +                        {7L, "0000000005", "T5", "0000000005", "S5"},
 +                        {8L, "0000000006", "T6", "0000000006", "S6"}})
 +                .close();
 +    }
 +
 +    /** Tests a simple command that is defined in Phoenix's extended SQL parser. 
 +     * @throws Exception */
 +    @Ignore
 +    @Test public void testCommit() throws Exception {
 +        start(false).sql("commit").execute();
 +    }
 +
 +    @Test public void testCreateView() throws Exception {
 +        start(false).sql("create view v as select * from (values (1, 'a'), (2, 'b')) as t(x, y)").execute();
 +    }
 +
 +    @Ignore // CALCITE-1045
 +    @Test public void testConnectJoinHsqldb() throws Exception {
 +        final Start start = new Start(getConnectionProps(false)) {
 +            @Override
 +            Connection createConnection() throws Exception {
 +                return connectWithHsqldbUsingModel(props);
 +            }
 +        };
 +        start.sql("select the_year, quantity as q, (select count(*) cnt \n"
 +            + "from \"foodmart\".\"time_by_day\" t where t.\"the_year\" = c.the_year)\n"
 +            + "from " + JOIN_ORDER_TABLE_FULL_NAME + " c")
 +            .explainIs("EnumerableCalc(expr#0..8=[{inputs}], THE_YEAR=[$t6], Q=[$t4], EXPR$2=[$t8])\n" +
 +                       "  EnumerableJoin(condition=[=($6, $7)], joinType=[left])\n" +
 +                       "    PhoenixToEnumerableConverter\n" +
 +                       "      PhoenixTableScan(table=[[phoenix, Join, OrderTable]])\n" +
 +                       "    EnumerableAggregate(group=[{0}], agg#0=[SINGLE_VALUE($1)])\n" +
 +                       "      EnumerableAggregate(group=[{0}], CNT=[COUNT()])\n" +
 +                       "        EnumerableJoin(condition=[=($0, $11)], joinType=[inner])\n" +
 +                       "          PhoenixToEnumerableConverter\n" +
 +                       "            PhoenixServerAggregate(group=[{6}], isOrdered=[false])\n" +
 +                       "              PhoenixTableScan(table=[[phoenix, Join, OrderTable]])\n" +
 +                       "          JdbcToEnumerableConverter\n" +
 +                       "            JdbcProject(time_id=[$0], th

<TRUNCATED>

[02/50] [abbrv] phoenix git commit: PHOENIX-2520 Create DDL property for metadata update frequency

Posted by ma...@apache.org.
PHOENIX-2520 Create DDL property for metadata update frequency


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

Branch: refs/heads/calcite
Commit: 59b336ec84b7cf5189bb2e67f07f9ef20da898d9
Parents: 3e5fa3e
Author: James Taylor <jt...@salesforce.com>
Authored: Sun Jan 17 11:24:26 2016 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Sun Jan 17 11:49:00 2016 -0800

----------------------------------------------------------------------
 .../org/apache/phoenix/rpc/UpdateCacheIT.java   |  92 ++++++++++----
 .../phoenix/rpc/UpdateCacheWithScnIT.java       |  17 ++-
 phoenix-core/src/main/antlr3/PhoenixSQL.g       |   3 +-
 .../apache/phoenix/compile/FromCompiler.java    |   2 +-
 .../apache/phoenix/compile/JoinCompiler.java    |   2 +-
 .../compile/TupleProjectionCompiler.java        |   4 +-
 .../apache/phoenix/compile/UnionCompiler.java   |   2 +-
 .../coprocessor/MetaDataEndpointImpl.java       |  13 +-
 .../phoenix/coprocessor/MetaDataProtocol.java   |   2 +-
 .../coprocessor/generated/PTableProtos.java     | 104 +++++++++++++--
 .../apache/phoenix/jdbc/PhoenixConnection.java  |   6 +-
 .../phoenix/jdbc/PhoenixDatabaseMetaData.java   |   3 +
 .../query/ConnectionQueryServicesImpl.java      |  43 +++----
 .../query/ConnectionlessQueryServicesImpl.java  |   4 +-
 .../query/DelegateConnectionQueryServices.java  |   4 +-
 .../apache/phoenix/query/MetaDataMutated.java   |   2 +-
 .../apache/phoenix/query/QueryConstants.java    |   4 +-
 .../apache/phoenix/schema/DelegateTable.java    |   6 +
 .../apache/phoenix/schema/MetaDataClient.java   | 123 ++++++++++++------
 .../org/apache/phoenix/schema/PMetaData.java    |   3 +-
 .../apache/phoenix/schema/PMetaDataImpl.java    |  15 ++-
 .../java/org/apache/phoenix/schema/PTable.java  |   1 +
 .../org/apache/phoenix/schema/PTableImpl.java   |  65 ++++++----
 .../org/apache/phoenix/schema/PTableRef.java    |  17 ++-
 .../apache/phoenix/schema/TableProperty.java    |  26 ++++
 .../phoenix/execute/CorrelatePlanTest.java      |   4 +-
 .../phoenix/schema/PMetaDataImplTest.java       | 125 ++++++++++++-------
 phoenix-protocol/src/main/PTable.proto          |   1 +
 28 files changed, 487 insertions(+), 206 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/59b336ec/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheIT.java b/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheIT.java
index 13ed8aa..20a3c48 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheIT.java
@@ -49,8 +49,8 @@ import org.apache.phoenix.schema.types.PVarchar;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.TestUtil;
-import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.mockito.Mockito;
@@ -73,48 +73,86 @@ public class UpdateCacheIT extends BaseHBaseManagedTimeIT {
         setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
     }
     
-    @Before
-    public void setUp() throws SQLException {
-        ensureTableCreated(getUrl(), MUTABLE_INDEX_DATA_TABLE);
-        ensureTableCreated(getUrl(), TRANSACTIONAL_DATA_TABLE);
-    }
-
-    private static void setupSystemTable(Long scn) throws SQLException {
+    private static void setupSystemTable(String fullTableName) throws SQLException {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        if (scn != null) {
-            props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(scn));
-        }
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
             conn.createStatement().execute(
-            "create table " + QueryConstants.SYSTEM_SCHEMA_NAME + QueryConstants.NAME_SEPARATOR + MUTABLE_INDEX_DATA_TABLE + TEST_TABLE_SCHEMA);
+            "create table " + fullTableName + TEST_TABLE_SCHEMA);
         }
     }
     
     @Test
     public void testUpdateCacheForTxnTable() throws Exception {
-        helpTestUpdateCache(true, false, null);
+        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + TRANSACTIONAL_DATA_TABLE;
+        ensureTableCreated(getUrl(), TRANSACTIONAL_DATA_TABLE);
+        helpTestUpdateCache(fullTableName, null, new int[] {1, 1});
     }
     
     @Test
     public void testUpdateCacheForNonTxnTable() throws Exception {
-        helpTestUpdateCache(false, false, null);
+        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + MUTABLE_INDEX_DATA_TABLE;
+        ensureTableCreated(getUrl(), MUTABLE_INDEX_DATA_TABLE);
+        helpTestUpdateCache(fullTableName, null, new int[] {1, 3});
     }
 	
     @Test
     public void testUpdateCacheForNonTxnSystemTable() throws Exception {
-        helpTestUpdateCache(false, true, null);
+        String fullTableName = QueryConstants.SYSTEM_SCHEMA_NAME + QueryConstants.NAME_SEPARATOR + MUTABLE_INDEX_DATA_TABLE;
+        setupSystemTable(fullTableName);
+        helpTestUpdateCache(fullTableName, null, new int[] {0, 0});
+    }
+    
+    @Test
+    public void testUpdateCacheForNeverUpdatedTable() throws Exception {
+        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + MUTABLE_INDEX_DATA_TABLE;
+        ensureTableCreated(getUrl(), MUTABLE_INDEX_DATA_TABLE);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.createStatement().execute(
+            "alter table " + fullTableName + " SET UPDATE_CACHE_FREQUENCY=NEVER");
+        }
+        helpTestUpdateCache(fullTableName, null, new int[] {0, 0});
+    }
+    
+    @Test
+    public void testUpdateCacheForAlwaysUpdatedTable() throws Exception {
+        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + MUTABLE_INDEX_DATA_TABLE;
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.createStatement().execute("CREATE TABLE " + fullTableName + TEST_TABLE_SCHEMA + " UPDATE_CACHE_FREQUENCY=always");
+        }
+        helpTestUpdateCache(fullTableName, null, new int[] {1, 3});
+    }
+    
+    @Test
+    public void testUpdateCacheForTimeLimitedUpdateTable() throws Exception {
+        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + MUTABLE_INDEX_DATA_TABLE;
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.createStatement().execute("CREATE TABLE " + fullTableName + TEST_TABLE_SCHEMA + " UPDATE_CACHE_FREQUENCY=" + 10000);
+        }
+        helpTestUpdateCache(fullTableName, null, new int[] {0, 0});
+        Thread.sleep(10000);
+        helpTestUpdateCache(fullTableName, null, new int[] {1, 0});
+    }
+    
+    @Test
+    public void testUpdateCacheForChangingUpdateTable() throws Exception {
+        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + MUTABLE_INDEX_DATA_TABLE;
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.createStatement().execute("CREATE TABLE " + fullTableName + TEST_TABLE_SCHEMA + " UPDATE_CACHE_FREQUENCY=never");
+        }
+        helpTestUpdateCache(fullTableName, null, new int[] {0, 0});
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.createStatement().execute("ALTER TABLE " + fullTableName + " SET UPDATE_CACHE_FREQUENCY=ALWAYS");
+        }
+        helpTestUpdateCache(fullTableName, null, new int[] {1, 3});
     }
     
-	public static void helpTestUpdateCache(boolean isTransactional, boolean isSystem, Long scn) throws Exception {
-	    String tableName = isTransactional ? TRANSACTIONAL_DATA_TABLE : MUTABLE_INDEX_DATA_TABLE;
-	    String schemaName;
-	    if (isSystem) {
-	        setupSystemTable(scn);
-	        schemaName = QueryConstants.SYSTEM_SCHEMA_NAME;
-	    } else {
-	        schemaName = INDEX_DATA_SCHEMA;
-	    }
-	    String fullTableName = schemaName + QueryConstants.NAME_SEPARATOR + tableName;
+	public static void helpTestUpdateCache(String fullTableName, Long scn, int[] expectedRPCs) throws Exception {
+	    String tableName = SchemaUtil.getTableNameFromFullName(fullTableName);
+	    String schemaName = SchemaUtil.getSchemaNameFromFullName(fullTableName);
 		String selectSql = "SELECT * FROM "+fullTableName;
 		// use a spyed ConnectionQueryServices so we can verify calls to getTable
 		ConnectionQueryServices connectionQueryServices = Mockito.spy(driver.getConnectionQueryServices(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)));
@@ -136,7 +174,7 @@ public class UpdateCacheIT extends BaseHBaseManagedTimeIT {
 			TestUtil.setRowKeyColumns(stmt, 3);
 			stmt.execute();
 			conn.commit();
-            int numUpsertRpcs = isSystem ? 0 : 1; 
+            int numUpsertRpcs = expectedRPCs[0];
 			// verify only 0 or 1 rpc to fetch table metadata, 
             verify(connectionQueryServices, times(numUpsertRpcs)).getTable((PName)isNull(), eq(PVarchar.INSTANCE.toBytes(schemaName)), eq(PVarchar.INSTANCE.toBytes(tableName)), anyLong(), anyLong());
             reset(connectionQueryServices);
@@ -169,7 +207,7 @@ public class UpdateCacheIT extends BaseHBaseManagedTimeIT {
             // for non-transactional tables with a scn : verify *only* one rpc occurs
             // for transactional tables : verify *only* one rpc occurs
 	        // for non-transactional, system tables : verify no rpc occurs
-            int numRpcs = isSystem ? 0 : (isTransactional || scn!=null ? 1 : 3); 
+            int numRpcs = expectedRPCs[1]; 
             verify(connectionQueryServices, times(numRpcs)).getTable((PName)isNull(), eq(PVarchar.INSTANCE.toBytes(schemaName)), eq(PVarchar.INSTANCE.toBytes(tableName)), anyLong(), anyLong());
 		}
         finally {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/59b336ec/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheWithScnIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheWithScnIT.java b/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheWithScnIT.java
index 5ff2fb0..04f751b 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheWithScnIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheWithScnIT.java
@@ -17,25 +17,22 @@
  */
 package org.apache.phoenix.rpc;
 
+import static org.apache.phoenix.util.TestUtil.INDEX_DATA_SCHEMA;
 import static org.apache.phoenix.util.TestUtil.MUTABLE_INDEX_DATA_TABLE;
 
 import org.apache.phoenix.end2end.BaseClientManagedTimeIT;
-import org.junit.Before;
+import org.apache.phoenix.query.QueryConstants;
 import org.junit.Test;
 
 public class UpdateCacheWithScnIT extends BaseClientManagedTimeIT {
 	
-	protected long ts;
-
-	@Before
-	public void initTable() throws Exception {
-		ts = nextTimestamp();
-		ensureTableCreated(getUrl(), MUTABLE_INDEX_DATA_TABLE, ts);
-	}
-	
 	@Test
 	public void testUpdateCacheWithScn() throws Exception {
-		UpdateCacheIT.helpTestUpdateCache(false, false, ts+2);
+        long ts = nextTimestamp();
+        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + MUTABLE_INDEX_DATA_TABLE;
+        ensureTableCreated(getUrl(), MUTABLE_INDEX_DATA_TABLE, ts);
+        // FIXME: given that the scn is advancing in the test, why aren't there more RPCs?
+		UpdateCacheIT.helpTestUpdateCache(fullTableName, ts+2, new int[] {1, 1});
 	}
 
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/59b336ec/phoenix-core/src/main/antlr3/PhoenixSQL.g
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/antlr3/PhoenixSQL.g b/phoenix-core/src/main/antlr3/PhoenixSQL.g
index 18a153e..23f7e8f 100644
--- a/phoenix-core/src/main/antlr3/PhoenixSQL.g
+++ b/phoenix-core/src/main/antlr3/PhoenixSQL.g
@@ -509,7 +509,8 @@ fam_prop_name returns [PropertyName ret]
     ;
     
 prop_value returns [Object ret]
-    :   l=literal { $ret = l.getValue(); }
+    :   v=identifier { $ret = v; }
+    |   l=literal { $ret = l.getValue(); }
     ;
     
 column_name returns [ColumnName ret]

http://git-wip-us.apache.org/repos/asf/phoenix/blob/59b336ec/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
index 0828b94..9b2c460 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
@@ -652,7 +652,7 @@ public class FromCompiler {
                     PTableType.SUBQUERY, null, MetaDataProtocol.MIN_TABLE_TIMESTAMP, PTable.INITIAL_SEQ_NUM,
                     null, null, columns, null, null, Collections.<PTable>emptyList(),
                     false, Collections.<PName>emptyList(), null, null, false, false, false, null,
-                    null, null, false, false);
+                    null, null, false, false, 0);
 
             String alias = subselectNode.getAlias();
             TableRef tableRef = new TableRef(alias, t, MetaDataProtocol.MIN_TABLE_TIMESTAMP, false);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/59b336ec/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
index b55e4aa..b64b9b7 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
@@ -1302,7 +1302,7 @@ public class JoinCompiler {
                 left.getBucketNum(), merged,left.getParentSchemaName(), left.getParentTableName(), left.getIndexes(),
                 left.isImmutableRows(), Collections.<PName>emptyList(), null, null, PTable.DEFAULT_DISABLE_WAL,
                 left.isMultiTenant(), left.getStoreNulls(), left.getViewType(), left.getViewIndexId(), left.getIndexType(),
-                left.rowKeyOrderOptimizable(), left.isTransactional());
+                left.rowKeyOrderOptimizable(), left.isTransactional(), left.getUpdateCacheFrequency());
     }
 
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/59b336ec/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
index 551b05c..0fc6d74 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
@@ -152,7 +152,7 @@ public class TupleProjectionCompiler {
                 table.getBucketNum(), projectedColumns, table.getParentSchemaName(),
                 table.getParentName(), table.getIndexes(), table.isImmutableRows(), Collections.<PName>emptyList(), null, null,
                 table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(),
-                table.getIndexType(), table.rowKeyOrderOptimizable(), table.isTransactional());
+                table.getIndexType(), table.rowKeyOrderOptimizable(), table.isTransactional(), table.getUpdateCacheFrequency());
     }
 
     public static PTable createProjectedTable(TableRef tableRef, List<ColumnRef> sourceColumnRefs, boolean retainPKColumns) throws SQLException {
@@ -179,7 +179,7 @@ public class TupleProjectionCompiler {
                     retainPKColumns ? table.getBucketNum() : null, projectedColumns, null,
                     null, Collections.<PTable>emptyList(), table.isImmutableRows(), Collections.<PName>emptyList(), null, null,
                     table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(),
-                    null, table.rowKeyOrderOptimizable(), table.isTransactional());
+                    null, table.rowKeyOrderOptimizable(), table.isTransactional(), table.getUpdateCacheFrequency());
     }
 
     // For extracting column references from single select statement

http://git-wip-us.apache.org/repos/asf/phoenix/blob/59b336ec/phoenix-core/src/main/java/org/apache/phoenix/compile/UnionCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/UnionCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/UnionCompiler.java
index 298303d..3bc1e37 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/UnionCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/UnionCompiler.java
@@ -82,7 +82,7 @@ public class UnionCompiler {
         PTable tempTable = PTableImpl.makePTable(statement.getConnection().getTenantId(), UNION_SCHEMA_NAME, UNION_TABLE_NAME, 
                 PTableType.SUBQUERY, null, HConstants.LATEST_TIMESTAMP, scn == null ? HConstants.LATEST_TIMESTAMP : scn, null, null,
                         projectedColumns, null, null, null,
-                        true, null, null, null, true, true, true, null, null, null, false, false);
+                        true, null, null, null, true, true, true, null, null, null, false, false, 0);
         TableRef tableRef = new TableRef(null, tempTable, 0, false);
         return tableRef;
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/59b336ec/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
----------------------------------------------------------------------
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 8c905ba..9887e7b 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
@@ -62,6 +62,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_TYPE_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TENANT_ID_INDEX;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TRANSACTIONAL_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TYPE_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.UPDATE_CACHE_FREQUENCY_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_CONSTANT_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_INDEX_ID_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_STATEMENT_BYTES;
@@ -242,6 +243,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     private static final KeyValue BASE_COLUMN_COUNT_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, PhoenixDatabaseMetaData.BASE_COLUMN_COUNT_BYTES);
     private static final KeyValue ROW_KEY_ORDER_OPTIMIZABLE_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, ROW_KEY_ORDER_OPTIMIZABLE_BYTES);
     private static final KeyValue TRANSACTIONAL_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, TRANSACTIONAL_BYTES);
+    private static final KeyValue UPDATE_CACHE_FREQUENCY_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, UPDATE_CACHE_FREQUENCY_BYTES);
     
     private static final List<KeyValue> TABLE_KV_COLUMNS = Arrays.<KeyValue>asList(
             EMPTY_KEYVALUE_KV,
@@ -264,7 +266,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             STORE_NULLS_KV,
             BASE_COLUMN_COUNT_KV,
             ROW_KEY_ORDER_OPTIMIZABLE_KV,
-            TRANSACTIONAL_KV
+            TRANSACTIONAL_KV,
+            UPDATE_CACHE_FREQUENCY_KV
             );
     static {
         Collections.sort(TABLE_KV_COLUMNS, KeyValue.COMPARATOR);
@@ -289,6 +292,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     private static final int BASE_COLUMN_COUNT_INDEX = TABLE_KV_COLUMNS.indexOf(BASE_COLUMN_COUNT_KV);
     private static final int ROW_KEY_ORDER_OPTIMIZABLE_INDEX = TABLE_KV_COLUMNS.indexOf(ROW_KEY_ORDER_OPTIMIZABLE_KV);
     private static final int TRANSACTIONAL_INDEX = TABLE_KV_COLUMNS.indexOf(TRANSACTIONAL_KV);
+    private static final int UPDATE_CACHE_FREQUENCY_INDEX = TABLE_KV_COLUMNS.indexOf(UPDATE_CACHE_FREQUENCY_KV);
 
     // KeyValues for Column
     private static final KeyValue DECIMAL_DIGITS_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, DECIMAL_DIGITS_BYTES);
@@ -806,6 +810,10 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             baseColumnCountKv.getValueOffset(), SortOrder.getDefault());
         Cell rowKeyOrderOptimizableKv = tableKeyValues[ROW_KEY_ORDER_OPTIMIZABLE_INDEX];
         boolean rowKeyOrderOptimizable = rowKeyOrderOptimizableKv == null ? false : Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(rowKeyOrderOptimizableKv.getValueArray(), rowKeyOrderOptimizableKv.getValueOffset(), rowKeyOrderOptimizableKv.getValueLength()));
+        Cell updateCacheFrequencyKv = tableKeyValues[UPDATE_CACHE_FREQUENCY_INDEX];
+        long updateCacheFrequency = updateCacheFrequencyKv == null ? 0 :
+            PLong.INSTANCE.getCodec().decodeLong(updateCacheFrequencyKv.getValueArray(),
+                    updateCacheFrequencyKv.getValueOffset(), SortOrder.getDefault());
 
         List<PColumn> columns = Lists.newArrayListWithExpectedSize(columnCount);
         List<PTable> indexes = new ArrayList<PTable>();
@@ -850,7 +858,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         return PTableImpl.makePTable(tenantId, schemaName, tableName, tableType, indexState, timeStamp,
             tableSeqNum, pkName, saltBucketNum, columns, tableType == INDEX ? schemaName : null,
             tableType == INDEX ? dataTableName : null, indexes, isImmutableRows, physicalTables, defaultFamilyName, viewStatement,
-            disableWAL, multiTenant, storeNulls, viewType, viewIndexId, indexType, stats, baseColumnCount, rowKeyOrderOptimizable, transactional);
+            disableWAL, multiTenant, storeNulls, viewType, viewIndexId, indexType, rowKeyOrderOptimizable, transactional, updateCacheFrequency,
+            stats, baseColumnCount);
     }
 
     private PFunction getFunction(RegionScanner scanner, final boolean isReplace, long clientTimeStamp, List<Mutation> deleteMutationsForReplace)

http://git-wip-us.apache.org/repos/asf/phoenix/blob/59b336ec/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
index b7b936e..a704e22 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
@@ -73,7 +73,7 @@ public abstract class MetaDataProtocol extends MetaDataService {
     public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_3_0 = MIN_TABLE_TIMESTAMP + 7;
     public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_5_0 = MIN_TABLE_TIMESTAMP + 8;
     public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_6_0 = MIN_TABLE_TIMESTAMP + 9;
-    public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 = MIN_TABLE_TIMESTAMP + 11;
+    public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 = MIN_TABLE_TIMESTAMP + 13;
     // MIN_SYSTEM_TABLE_TIMESTAMP needs to be set to the max of all the MIN_SYSTEM_TABLE_TIMESTAMP_* constants
     public static final long MIN_SYSTEM_TABLE_TIMESTAMP = MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0;
     // TODO: pare this down to minimum, as we don't need duplicates for both table and column errors, nor should we need

http://git-wip-us.apache.org/repos/asf/phoenix/blob/59b336ec/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java
index 7e71cd9..be8d7e2 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java
@@ -3228,6 +3228,16 @@ public final class PTableProtos {
      * <code>optional bool transactional = 27;</code>
      */
     boolean getTransactional();
+
+    // optional int64 updateCacheFrequency = 28;
+    /**
+     * <code>optional int64 updateCacheFrequency = 28;</code>
+     */
+    boolean hasUpdateCacheFrequency();
+    /**
+     * <code>optional int64 updateCacheFrequency = 28;</code>
+     */
+    long getUpdateCacheFrequency();
   }
   /**
    * Protobuf type {@code PTable}
@@ -3433,6 +3443,11 @@ public final class PTableProtos {
               transactional_ = input.readBool();
               break;
             }
+            case 224: {
+              bitField0_ |= 0x00800000;
+              updateCacheFrequency_ = input.readInt64();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -4011,6 +4026,22 @@ public final class PTableProtos {
       return transactional_;
     }
 
+    // optional int64 updateCacheFrequency = 28;
+    public static final int UPDATECACHEFREQUENCY_FIELD_NUMBER = 28;
+    private long updateCacheFrequency_;
+    /**
+     * <code>optional int64 updateCacheFrequency = 28;</code>
+     */
+    public boolean hasUpdateCacheFrequency() {
+      return ((bitField0_ & 0x00800000) == 0x00800000);
+    }
+    /**
+     * <code>optional int64 updateCacheFrequency = 28;</code>
+     */
+    public long getUpdateCacheFrequency() {
+      return updateCacheFrequency_;
+    }
+
     private void initFields() {
       schemaNameBytes_ = com.google.protobuf.ByteString.EMPTY;
       tableNameBytes_ = com.google.protobuf.ByteString.EMPTY;
@@ -4039,6 +4070,7 @@ public final class PTableProtos {
       baseColumnCount_ = 0;
       rowKeyOrderOptimizable_ = false;
       transactional_ = false;
+      updateCacheFrequency_ = 0L;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -4187,6 +4219,9 @@ public final class PTableProtos {
       if (((bitField0_ & 0x00400000) == 0x00400000)) {
         output.writeBool(27, transactional_);
       }
+      if (((bitField0_ & 0x00800000) == 0x00800000)) {
+        output.writeInt64(28, updateCacheFrequency_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -4309,6 +4344,10 @@ public final class PTableProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeBoolSize(27, transactional_);
       }
+      if (((bitField0_ & 0x00800000) == 0x00800000)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt64Size(28, updateCacheFrequency_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -4455,6 +4494,11 @@ public final class PTableProtos {
         result = result && (getTransactional()
             == other.getTransactional());
       }
+      result = result && (hasUpdateCacheFrequency() == other.hasUpdateCacheFrequency());
+      if (hasUpdateCacheFrequency()) {
+        result = result && (getUpdateCacheFrequency()
+            == other.getUpdateCacheFrequency());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -4576,6 +4620,10 @@ public final class PTableProtos {
         hash = (37 * hash) + TRANSACTIONAL_FIELD_NUMBER;
         hash = (53 * hash) + hashBoolean(getTransactional());
       }
+      if (hasUpdateCacheFrequency()) {
+        hash = (37 * hash) + UPDATECACHEFREQUENCY_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getUpdateCacheFrequency());
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -4754,6 +4802,8 @@ public final class PTableProtos {
         bitField0_ = (bitField0_ & ~0x02000000);
         transactional_ = false;
         bitField0_ = (bitField0_ & ~0x04000000);
+        updateCacheFrequency_ = 0L;
+        bitField0_ = (bitField0_ & ~0x08000000);
         return this;
       }
 
@@ -4906,6 +4956,10 @@ public final class PTableProtos {
           to_bitField0_ |= 0x00400000;
         }
         result.transactional_ = transactional_;
+        if (((from_bitField0_ & 0x08000000) == 0x08000000)) {
+          to_bitField0_ |= 0x00800000;
+        }
+        result.updateCacheFrequency_ = updateCacheFrequency_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -5081,6 +5135,9 @@ public final class PTableProtos {
         if (other.hasTransactional()) {
           setTransactional(other.getTransactional());
         }
+        if (other.hasUpdateCacheFrequency()) {
+          setUpdateCacheFrequency(other.getUpdateCacheFrequency());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -6784,6 +6841,39 @@ public final class PTableProtos {
         return this;
       }
 
+      // optional int64 updateCacheFrequency = 28;
+      private long updateCacheFrequency_ ;
+      /**
+       * <code>optional int64 updateCacheFrequency = 28;</code>
+       */
+      public boolean hasUpdateCacheFrequency() {
+        return ((bitField0_ & 0x08000000) == 0x08000000);
+      }
+      /**
+       * <code>optional int64 updateCacheFrequency = 28;</code>
+       */
+      public long getUpdateCacheFrequency() {
+        return updateCacheFrequency_;
+      }
+      /**
+       * <code>optional int64 updateCacheFrequency = 28;</code>
+       */
+      public Builder setUpdateCacheFrequency(long value) {
+        bitField0_ |= 0x08000000;
+        updateCacheFrequency_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional int64 updateCacheFrequency = 28;</code>
+       */
+      public Builder clearUpdateCacheFrequency() {
+        bitField0_ = (bitField0_ & ~0x08000000);
+        updateCacheFrequency_ = 0L;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:PTable)
     }
 
@@ -6830,7 +6920,7 @@ public final class PTableProtos {
       "leStats\022\013\n\003key\030\001 \002(\014\022\016\n\006values\030\002 \003(\014\022\033\n\023" +
       "guidePostsByteCount\030\003 \001(\003\022\025\n\rkeyBytesCou",
       "nt\030\004 \001(\003\022\027\n\017guidePostsCount\030\005 \001(\005\022!\n\013pGu" +
-      "idePosts\030\006 \001(\0132\014.PGuidePosts\"\206\005\n\006PTable\022" +
+      "idePosts\030\006 \001(\0132\014.PGuidePosts\"\244\005\n\006PTable\022" +
       "\027\n\017schemaNameBytes\030\001 \002(\014\022\026\n\016tableNameByt" +
       "es\030\002 \002(\014\022\036\n\ttableType\030\003 \002(\0162\013.PTableType" +
       "\022\022\n\nindexState\030\004 \001(\t\022\026\n\016sequenceNumber\030\005" +
@@ -6846,11 +6936,11 @@ public final class PTableProtos {
       "wIndexId\030\025 \001(\005\022\021\n\tindexType\030\026 \001(\014\022\026\n\016sta" +
       "tsTimeStamp\030\027 \001(\003\022\022\n\nstoreNulls\030\030 \001(\010\022\027\n" +
       "\017baseColumnCount\030\031 \001(\005\022\036\n\026rowKeyOrderOpt" +
-      "imizable\030\032 \001(\010\022\025\n\rtransactional\030\033 \001(\010*A\n" +
-      "\nPTableType\022\n\n\006SYSTEM\020\000\022\010\n\004USER\020\001\022\010\n\004VIE" +
-      "W\020\002\022\t\n\005INDEX\020\003\022\010\n\004JOIN\020\004B@\n(org.apache.p",
-      "hoenix.coprocessor.generatedB\014PTableProt" +
-      "osH\001\210\001\001\240\001\001"
+      "imizable\030\032 \001(\010\022\025\n\rtransactional\030\033 \001(\010\022\034\n" +
+      "\024updateCacheFrequency\030\034 \001(\003*A\n\nPTableTyp" +
+      "e\022\n\n\006SYSTEM\020\000\022\010\n\004USER\020\001\022\010\n\004VIEW\020\002\022\t\n\005IND",
+      "EX\020\003\022\010\n\004JOIN\020\004B@\n(org.apache.phoenix.cop" +
+      "rocessor.generatedB\014PTableProtosH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -6874,7 +6964,7 @@ public final class PTableProtos {
           internal_static_PTable_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_PTable_descriptor,
-              new java.lang.String[] { "SchemaNameBytes", "TableNameBytes", "TableType", "IndexState", "SequenceNumber", "TimeStamp", "PkNameBytes", "BucketNum", "Columns", "Indexes", "IsImmutableRows", "GuidePosts", "DataTableNameBytes", "DefaultFamilyName", "DisableWAL", "MultiTenant", "ViewType", "ViewStatement", "PhysicalNames", "TenantId", "ViewIndexId", "IndexType", "StatsTimeStamp", "StoreNulls", "BaseColumnCount", "RowKeyOrderOptimizable", "Transactional", });
+              new java.lang.String[] { "SchemaNameBytes", "TableNameBytes", "TableType", "IndexState", "SequenceNumber", "TimeStamp", "PkNameBytes", "BucketNum", "Columns", "Indexes", "IsImmutableRows", "GuidePosts", "DataTableNameBytes", "DefaultFamilyName", "DisableWAL", "MultiTenant", "ViewType", "ViewStatement", "PhysicalNames", "TenantId", "ViewIndexId", "IndexType", "StatsTimeStamp", "StoreNulls", "BaseColumnCount", "RowKeyOrderOptimizable", "Transactional", "UpdateCacheFrequency", });
           return null;
         }
       };

http://git-wip-us.apache.org/repos/asf/phoenix/blob/59b336ec/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
index 208e874..82bf31a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
@@ -913,11 +913,11 @@ public class PhoenixConnection implements Connection, MetaDataMutated, SQLClosea
     }
 
     @Override
-    public PMetaData addColumn(PName tenantId, String tableName, List<PColumn> columns, long tableTimeStamp, long tableSeqNum, boolean isImmutableRows, boolean isWalDisabled, boolean isMultitenant, boolean storeNulls, boolean isTransactional, long resolvedTime)
+    public PMetaData addColumn(PName tenantId, String tableName, List<PColumn> columns, long tableTimeStamp, long tableSeqNum, boolean isImmutableRows, boolean isWalDisabled, boolean isMultitenant, boolean storeNulls, boolean isTransactional, long updateCacheFrequency, long resolvedTime)
             throws SQLException {
-        metaData = metaData.addColumn(tenantId, tableName, columns, tableTimeStamp, tableSeqNum, isImmutableRows, isWalDisabled, isMultitenant, storeNulls, isTransactional, resolvedTime);
+        metaData = metaData.addColumn(tenantId, tableName, columns, tableTimeStamp, tableSeqNum, isImmutableRows, isWalDisabled, isMultitenant, storeNulls, isTransactional, updateCacheFrequency, resolvedTime);
         //Cascade through to connectionQueryServices too
-        getQueryServices().addColumn(tenantId, tableName, columns, tableTimeStamp, tableSeqNum, isImmutableRows, isWalDisabled, isMultitenant, storeNulls, isTransactional, resolvedTime);
+        getQueryServices().addColumn(tenantId, tableName, columns, tableTimeStamp, tableSeqNum, isImmutableRows, isWalDisabled, isMultitenant, storeNulls, isTransactional, updateCacheFrequency, resolvedTime);
         return metaData;
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/59b336ec/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
index 869ba19..fabd949 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
@@ -285,6 +285,9 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData {
     public static final String TRANSACTIONAL = "TRANSACTIONAL";
     public static final byte[] TRANSACTIONAL_BYTES = Bytes.toBytes(TRANSACTIONAL);
 
+    public static final String UPDATE_CACHE_FREQUENCY = "UPDATE_CACHE_FREQUENCY";
+    public static final byte[] UPDATE_CACHE_FREQUENCY_BYTES = Bytes.toBytes(UPDATE_CACHE_FREQUENCY);
+
     public static final String ASYNC_CREATED_DATE = "ASYNC_CREATED_DATE";
 
     private final PhoenixConnection connection;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/59b336ec/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 8ef3161..a246e63 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
@@ -630,12 +630,14 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
      }
 
     @Override
-    public PMetaData addColumn(final PName tenantId, final String tableName, final List<PColumn> columns, final long tableTimeStamp, final long tableSeqNum, final boolean isImmutableRows, final boolean isWalDisabled, final boolean isMultitenant, final boolean storeNulls, final boolean isTransactional, final long resolvedTime) throws SQLException {
+    public PMetaData addColumn(final PName tenantId, final String tableName, final List<PColumn> columns, final long tableTimeStamp, 
+            final long tableSeqNum, final boolean isImmutableRows, final boolean isWalDisabled, final boolean isMultitenant, 
+            final boolean storeNulls, final boolean isTransactional, final long updateCacheFrequency, final long resolvedTime) throws SQLException {
         return metaDataMutated(tenantId, tableName, tableSeqNum, new Mutator() {
             @Override
             public PMetaData mutate(PMetaData metaData) throws SQLException {
                 try {
-                    return metaData.addColumn(tenantId, tableName, columns, tableTimeStamp, tableSeqNum, isImmutableRows, isWalDisabled, isMultitenant, storeNulls, isTransactional, resolvedTime);
+                    return metaData.addColumn(tenantId, tableName, columns, tableTimeStamp, tableSeqNum, isImmutableRows, isWalDisabled, isMultitenant, storeNulls, isTransactional, updateCacheFrequency, resolvedTime);
                 } catch (TableNotFoundException e) {
                     // The DROP TABLE may have been processed first, so just ignore.
                     return metaData;
@@ -2410,17 +2412,20 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                 }
                                 if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_6_0) {
                                     columnsToAdd = PhoenixDatabaseMetaData.IS_ROW_TIMESTAMP + " " + PBoolean.INSTANCE.getSqlTypeName();
-                                    metaConnection = addColumn(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG,
-                                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_6_0, columnsToAdd, false);
+                                    metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+                                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_6_0, columnsToAdd);
                                 }
                                 if(currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0) {
-                                    columnsToAdd = PhoenixDatabaseMetaData.TRANSACTIONAL + " " + PBoolean.INSTANCE.getSqlTypeName();
-                                    metaConnection = addColumn(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG,
-                                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0, columnsToAdd, false);
-									// Drop old stats table so that new stats
-									// table
+                                    // Add these columns one at a time, each with different timestamps so that if folks have
+                                    // run the upgrade code already for a snapshot, we'll still enter this block (and do the
+                                    // parts we haven't yet done).
+                                    metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG, MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 2,
+                                            PhoenixDatabaseMetaData.TRANSACTIONAL + " " + PBoolean.INSTANCE.getSqlTypeName());
+                                    metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG, MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 1, 
+                                            PhoenixDatabaseMetaData.UPDATE_CACHE_FREQUENCY + " " + PLong.INSTANCE.getSqlTypeName());
+									// Drop old stats table so that new stats table is created
 									metaConnection = dropStatsTable(metaConnection,
-											MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 1);
+											MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0);
                                 }
                                 
                             }
@@ -2525,7 +2530,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
 	private PhoenixConnection dropStatsTable(PhoenixConnection oldMetaConnection, long timestamp)
 			throws SQLException, IOException {
 		Properties props = PropertiesUtil.deepCopy(oldMetaConnection.getClientInfo());
-		props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(timestamp));
+		props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(timestamp-1));
 		PhoenixConnection metaConnection = new PhoenixConnection(oldMetaConnection, this, props);
 		SQLException sqlE = null;
 		boolean wasCommit = metaConnection.getAutoCommit();
@@ -2555,23 +2560,9 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
 			}
 		}
 
-		HBaseAdmin admin = null;
-		try {
-			admin = getAdmin();
-			admin.disableTable(PhoenixDatabaseMetaData.SYSTEM_STATS_NAME_BYTES);
-			try {
-				admin.deleteTable(PhoenixDatabaseMetaData.SYSTEM_STATS_NAME_BYTES);
-			} catch (org.apache.hadoop.hbase.TableNotFoundException e) {
-				logger.debug("Stats table was not found during upgrade!!");
-			}
-		} finally {
-			if (admin != null)
-				admin.close();
-		}
 		oldMetaConnection = metaConnection;
 		props = PropertiesUtil.deepCopy(oldMetaConnection.getClientInfo());
-		props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB,
-				Long.toString(MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0));
+		props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(timestamp));
 		try {
 			metaConnection = new PhoenixConnection(oldMetaConnection, ConnectionQueryServicesImpl.this, props);
 		} finally {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/59b336ec/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
index 199b010..6cfb382 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
@@ -177,8 +177,8 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
 
     @Override
     public PMetaData addColumn(PName tenantId, String tableName, List<PColumn> columns, long tableTimeStamp,
-            long tableSeqNum, boolean isImmutableRows, boolean isWalDisabled, boolean isMultitenant, boolean storeNulls, boolean isTransactional, long resolvedTime) throws SQLException {
-        return metaData = metaData.addColumn(tenantId, tableName, columns, tableTimeStamp, tableSeqNum, isImmutableRows, isWalDisabled, isMultitenant, storeNulls, isTransactional, resolvedTime);
+            long tableSeqNum, boolean isImmutableRows, boolean isWalDisabled, boolean isMultitenant, boolean storeNulls, boolean isTransactional, long updateCacheFrequency, long resolvedTime) throws SQLException {
+        return metaData = metaData.addColumn(tenantId, tableName, columns, tableTimeStamp, tableSeqNum, isImmutableRows, isWalDisabled, isMultitenant, storeNulls, isTransactional, updateCacheFrequency, resolvedTime);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/59b336ec/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
index 84f3e74..9b721f8 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
@@ -87,8 +87,8 @@ public class DelegateConnectionQueryServices extends DelegateQueryServices imple
 
     @Override
     public PMetaData addColumn(PName tenantId, String tableName, List<PColumn> columns, long tableTimeStamp,
-            long tableSeqNum, boolean isImmutableRows, boolean isWalDisabled, boolean isMultitenant, boolean storeNulls, boolean isTransactional, long resolvedTime) throws SQLException {
-        return getDelegate().addColumn(tenantId, tableName, columns, tableTimeStamp, tableSeqNum, isImmutableRows, isWalDisabled, isMultitenant, storeNulls, isTransactional, resolvedTime);
+            long tableSeqNum, boolean isImmutableRows, boolean isWalDisabled, boolean isMultitenant, boolean storeNulls, boolean isTransactional, long updateCacheFrequency, long resolvedTime) throws SQLException {
+        return getDelegate().addColumn(tenantId, tableName, columns, tableTimeStamp, tableSeqNum, isImmutableRows, isWalDisabled, isMultitenant, storeNulls, isTransactional, updateCacheFrequency, resolvedTime);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/59b336ec/phoenix-core/src/main/java/org/apache/phoenix/query/MetaDataMutated.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/MetaDataMutated.java b/phoenix-core/src/main/java/org/apache/phoenix/query/MetaDataMutated.java
index 8e7a70d..753b172 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/MetaDataMutated.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/MetaDataMutated.java
@@ -38,7 +38,7 @@ public interface MetaDataMutated {
     PMetaData addTable(PTable table, long resolvedTime) throws SQLException;
     PMetaData updateResolvedTimestamp(PTable table, long resolvedTimestamp) throws SQLException;
     PMetaData removeTable(PName tenantId, String tableName, String parentTableName, long tableTimeStamp) throws SQLException;
-    PMetaData addColumn(PName tenantId, String tableName, List<PColumn> columns, long tableTimeStamp, long tableSeqNum, boolean isImmutableRows, boolean isWalDisabled, boolean isMultitenant, boolean storeNulls, boolean isTransactional, long resolvedTime) throws SQLException;
+    PMetaData addColumn(PName tenantId, String tableName, List<PColumn> columns, long tableTimeStamp, long tableSeqNum, boolean isImmutableRows, boolean isWalDisabled, boolean isMultitenant, boolean storeNulls, boolean isTransactional, long updateCacheFrequency, long resolvedTime) throws SQLException;
     PMetaData removeColumn(PName tenantId, String tableName, List<PColumn> columnsToRemove, long tableTimeStamp, long tableSeqNum, long resolvedTime) throws SQLException;
     PMetaData addFunction(PFunction function) throws SQLException;
     PMetaData removeFunction(PName tenantId, String function, long functionTimeStamp) throws SQLException;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/59b336ec/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
index c1cb0c0..63d4e07 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
@@ -39,9 +39,9 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DEFAULT_COLUMN_FAM
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DEFAULT_VALUE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DISABLE_WAL;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.FUNCTION_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.GUIDE_POSTS_ROW_COUNT;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.GUIDE_POST_KEY;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.GUIDE_POSTS_ROW_COUNT;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IMMUTABLE_ROWS;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INCREMENT_BY;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP;
@@ -96,6 +96,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TRANSACTIONAL;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TYPE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TYPE_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TYPE_SEQUENCE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.UPDATE_CACHE_FREQUENCY;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_CONSTANT;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_INDEX_ID;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_STATEMENT;
@@ -257,6 +258,7 @@ public interface QueryConstants {
             // Column metadata (will be null for table row)
             IS_ROW_TIMESTAMP + " BOOLEAN, " +
             TRANSACTIONAL + " BOOLEAN," +
+            UPDATE_CACHE_FREQUENCY + " BIGINT," +
             "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + ","
             + TABLE_SCHEM + "," + TABLE_NAME + "," + COLUMN_NAME + "," + COLUMN_FAMILY + "))\n" +
             HConstants.VERSIONS + "=" + MetaDataProtocol.DEFAULT_MAX_META_DATA_VERSIONS + ",\n" +

http://git-wip-us.apache.org/repos/asf/phoenix/blob/59b336ec/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateTable.java
----------------------------------------------------------------------
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 7fb90a1..e7bf961 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
@@ -242,6 +242,7 @@ public class DelegateTable implements PTable {
         return delegate.isTransactional();
     }
 
+    @Override
     public int getBaseColumnCount() {
         return delegate.getBaseColumnCount();
     }
@@ -260,4 +261,9 @@ public class DelegateTable implements PTable {
     public String toString() {
         return delegate.toString();
     }
+
+    @Override
+    public long getUpdateCacheFrequency() {
+        return delegate.getUpdateCacheFrequency();
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/59b336ec/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 ee8fba3..ee212ed 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
@@ -76,6 +76,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_TYPE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TENANT_ID;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TRANSACTIONAL;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TYPE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.UPDATE_CACHE_FREQUENCY;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_CONSTANT;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_INDEX_ID;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_STATEMENT;
@@ -250,8 +251,9 @@ public class MetaDataClient {
             INDEX_TYPE + "," +
             STORE_NULLS + "," +
             BASE_COLUMN_COUNT + "," +
-            TRANSACTIONAL +
-            ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
+            TRANSACTIONAL + "," +
+            UPDATE_CACHE_FREQUENCY +
+            ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
     private static final String CREATE_LINK =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
             TENANT_ID + "," +
@@ -477,8 +479,8 @@ public class MetaDataClient {
         // Do not make rpc to getTable if 
         // 1. table is a system table
         // 2. table was already resolved as of that timestamp
-        if (table != null && !alwaysHitServer
-                && (systemTable || resolvedTimestamp == tableResolvedTimestamp)) {
+		if (table != null && !alwaysHitServer
+				&& (systemTable || resolvedTimestamp == tableResolvedTimestamp || connection.getMetaDataCache().getAge(tableRef) < table.getUpdateCacheFrequency())) {
             return new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS, QueryConstants.UNSET_TIMESTAMP, table);
         }
 
@@ -1649,7 +1651,7 @@ public class MetaDataClient {
             // Although unusual, it's possible to set a mapped VIEW as having immutable rows.
             // This tells Phoenix that you're managing the index maintenance yourself.
             if (tableType != PTableType.INDEX && (tableType != PTableType.VIEW || viewType == ViewType.MAPPED)) {
-                Boolean isImmutableRowsProp = (Boolean) tableProps.get(PTable.IS_IMMUTABLE_ROWS_PROP_NAME);
+                Boolean isImmutableRowsProp = (Boolean) TableProperty.IMMUTABLE_ROWS.getValue(tableProps);
                 if (isImmutableRowsProp == null) {
                     isImmutableRows = connection.getQueryServices().getProps().getBoolean(QueryServices.IMMUTABLE_ROWS_ATTRIB, QueryServicesOptions.DEFAULT_IMMUTABLE_ROWS);
                 } else {
@@ -1659,7 +1661,7 @@ public class MetaDataClient {
 
             // Can't set any of these on views or shared indexes on views
             if (tableType != PTableType.VIEW && indexId == null) {
-                saltBucketNum = (Integer) tableProps.get(PhoenixDatabaseMetaData.SALT_BUCKETS);
+                saltBucketNum = (Integer) TableProperty.SALT_BUCKETS.getValue(tableProps);
                 if (saltBucketNum != null) {
                     if (saltBucketNum < 0 || saltBucketNum > SaltingUtil.MAX_BUCKET_NUM) {
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.INVALID_BUCKET_NUM).build().buildException();
@@ -1680,16 +1682,21 @@ public class MetaDataClient {
             if (tableType != PTableType.INDEX && (tableType != PTableType.VIEW || viewType == ViewType.MAPPED)) {
                 Boolean multiTenantProp = (Boolean) tableProps.get(PhoenixDatabaseMetaData.MULTI_TENANT);
                 multiTenant = Boolean.TRUE.equals(multiTenantProp);
-                defaultFamilyName = (String)tableProps.get(PhoenixDatabaseMetaData.DEFAULT_COLUMN_FAMILY_NAME);
+                defaultFamilyName = (String)TableProperty.DEFAULT_COLUMN_FAMILY.getValue(tableProps);
             }
 
             boolean disableWAL = false;
-            Boolean disableWALProp = (Boolean) tableProps.get(PhoenixDatabaseMetaData.DISABLE_WAL);
+            Boolean disableWALProp = (Boolean) TableProperty.DISABLE_WAL.getValue(tableProps);
             if (disableWALProp != null) {
                 disableWAL = disableWALProp;
             }
+            long updateCacheFrequency = 0;
+            Long updateCacheFrequencyProp = (Long) TableProperty.UPDATE_CACHE_FREQUENCY.getValue(tableProps);
+            if (updateCacheFrequencyProp != null) {
+                updateCacheFrequency = updateCacheFrequencyProp;
+            }
 
-            Boolean storeNullsProp = (Boolean) tableProps.get(PhoenixDatabaseMetaData.STORE_NULLS);
+            Boolean storeNullsProp = (Boolean) TableProperty.STORE_NULLS.getValue(tableProps);
             if (storeNullsProp == null) {
                 if (parent == null) {
                     storeNulls = connection.getQueryServices().getProps().getBoolean(
@@ -1700,7 +1707,7 @@ public class MetaDataClient {
             } else {
                 storeNulls = storeNullsProp;
             }
-            Boolean transactionalProp = (Boolean) tableProps.get(PhoenixDatabaseMetaData.TRANSACTIONAL);
+            Boolean transactionalProp = (Boolean) TableProperty.TRANSACTIONAL.getValue(tableProps);
             if (transactionalProp != null && parent != null) {
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.ONLY_TABLE_MAY_BE_DECLARED_TRANSACTIONAL)
                 .setSchemaName(schemaName).setTableName(tableName)
@@ -1731,7 +1738,8 @@ public class MetaDataClient {
                 .build().buildException();
             }
             
-            
+            // Put potentially inferred value into tableProps as it's used by the createTable call below
+            // to determine which coprocessors to install on the new table.
             tableProps.put(PhoenixDatabaseMetaData.TRANSACTIONAL, transactional);
             if (transactional) {
                 // If TTL set, use Tephra TTL property name instead
@@ -2020,7 +2028,7 @@ public class MetaDataClient {
                         Collections.<PTable>emptyList(), isImmutableRows,
                         Collections.<PName>emptyList(), defaultFamilyName == null ? null :
                                 PNameFactory.newName(defaultFamilyName), null,
-                        Boolean.TRUE.equals(disableWAL), false, false, null, indexId, indexType, true, false);
+                        Boolean.TRUE.equals(disableWAL), false, false, null, indexId, indexType, true, false, 0);
                 connection.addTable(table, MetaDataProtocol.MIN_TABLE_TIMESTAMP);
             } else if (tableType == PTableType.INDEX && indexId == null) {
                 if (tableProps.get(HTableDescriptor.MAX_FILESIZE) == null) {
@@ -2123,6 +2131,7 @@ public class MetaDataClient {
                 tableUpsert.setInt(20, BASE_TABLE_BASE_COLUMN_COUNT);
             }
             tableUpsert.setBoolean(21, transactional);
+            tableUpsert.setLong(22, updateCacheFrequency);
             tableUpsert.execute();
 
             if (asyncCreatedDate != null) {
@@ -2185,7 +2194,7 @@ public class MetaDataClient {
                         PTable.INITIAL_SEQ_NUM, pkName == null ? null : PNameFactory.newName(pkName), saltBucketNum, columns,
                         dataTableName == null ? null : newSchemaName, dataTableName == null ? null : PNameFactory.newName(dataTableName), Collections.<PTable>emptyList(), isImmutableRows,
                         physicalNames, defaultFamilyName == null ? null : PNameFactory.newName(defaultFamilyName), viewStatement, Boolean.TRUE.equals(disableWAL), multiTenant, storeNulls, viewType,
-                        indexId, indexType, rowKeyOrderOptimizable, transactional);
+                        indexId, indexType, rowKeyOrderOptimizable, transactional, updateCacheFrequency);
                 result = new MetaDataMutationResult(code, result.getMutationTime(), table, true);
                 addTableToCache(result);
                 return table;
@@ -2508,12 +2517,12 @@ public class MetaDataClient {
         return mutationCode;
     }
 
-    private  long incrementTableSeqNum(PTable table, PTableType expectedType, int columnCountDelta, Boolean isTransactional) throws SQLException {
-        return incrementTableSeqNum(table, expectedType, columnCountDelta, isTransactional, null, null, null, null);
+    private  long incrementTableSeqNum(PTable table, PTableType expectedType, int columnCountDelta, Boolean isTransactional, Long updateCacheFrequency) throws SQLException {
+        return incrementTableSeqNum(table, expectedType, columnCountDelta, isTransactional, updateCacheFrequency, null, null, null, null);
     }
 
     private long incrementTableSeqNum(PTable table, PTableType expectedType, int columnCountDelta,
-            Boolean isTransactional, Boolean isImmutableRows, Boolean disableWAL, Boolean isMultiTenant, Boolean storeNulls)
+            Boolean isTransactional, Long updateCacheFrequency, Boolean isImmutableRows, Boolean disableWAL, Boolean isMultiTenant, Boolean storeNulls)
             throws SQLException {
         String schemaName = table.getSchemaName().getString();
         String tableName = table.getTableName().getString();
@@ -2548,6 +2557,9 @@ public class MetaDataClient {
         if (isTransactional != null) {
             mutateBooleanProperty(tenantId, schemaName, tableName, TRANSACTIONAL, isTransactional);
         }
+        if (updateCacheFrequency != null) {
+            mutateLongProperty(tenantId, schemaName, tableName, UPDATE_CACHE_FREQUENCY, updateCacheFrequency);
+        }
         return seqNum;
     }
 
@@ -2567,6 +2579,22 @@ public class MetaDataClient {
         tableBoolUpsert.execute();
     }
 
+    private void mutateLongProperty(String tenantId, String schemaName, String tableName,
+            String propertyName, long propertyValue) throws SQLException {
+        String updatePropertySql = "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
+                        TENANT_ID + "," +
+                        TABLE_SCHEM + "," +
+                        TABLE_NAME + "," +
+                        propertyName +
+                        ") VALUES (?, ?, ?, ?)";
+        PreparedStatement tableBoolUpsert = connection.prepareStatement(updatePropertySql);
+        tableBoolUpsert.setString(1, tenantId);
+        tableBoolUpsert.setString(2, schemaName);
+        tableBoolUpsert.setString(3, tableName);
+        tableBoolUpsert.setLong(4, propertyValue);
+        tableBoolUpsert.execute();
+    }
+
     public MutationState addColumn(AddColumnStatement statement) throws SQLException {
         connection.rollback();
         boolean wasAutoCommit = connection.getAutoCommit();
@@ -2582,6 +2610,7 @@ public class MetaDataClient {
             Boolean disableWALProp = null;
             Boolean storeNullsProp = null;
             Boolean isTransactionalProp = null;
+            Long updateCacheFrequencyProp = null;
 
             ListMultimap<String,Pair<String,Object>> stmtProperties = statement.getProps();
             Map<String, List<Pair<String, Object>>> properties = new HashMap<>(stmtProperties.size());
@@ -2596,17 +2625,21 @@ public class MetaDataClient {
                 for (Pair<String, Object> prop : propsList) {
                     String propName = prop.getFirst();
                     if (TableProperty.isPhoenixTableProperty(propName)) {
-                        TableProperty.valueOf(propName).validate(true, !family.equals(QueryConstants.ALL_FAMILY_PROPERTIES_KEY), table.getType());
+                        TableProperty tableProp = TableProperty.valueOf(propName);
+                        tableProp.validate(true, !family.equals(QueryConstants.ALL_FAMILY_PROPERTIES_KEY), table.getType());
+                        Object value = tableProp.getValue(prop.getSecond());
                         if (propName.equals(PTable.IS_IMMUTABLE_ROWS_PROP_NAME)) {
-                            isImmutableRowsProp = (Boolean)prop.getSecond();
+                            isImmutableRowsProp = (Boolean)value;
                         } else if (propName.equals(PhoenixDatabaseMetaData.MULTI_TENANT)) {
-                            multiTenantProp = (Boolean)prop.getSecond();
+                            multiTenantProp = (Boolean)value;
                         } else if (propName.equals(DISABLE_WAL)) {
-                            disableWALProp = (Boolean)prop.getSecond();
+                            disableWALProp = (Boolean)value;
                         } else if (propName.equals(STORE_NULLS)) {
-                            storeNullsProp = (Boolean)prop.getSecond();
+                            storeNullsProp = (Boolean)value;
                         } else if (propName.equals(TRANSACTIONAL)) {
-                            isTransactionalProp = (Boolean)prop.getSecond();
+                            isTransactionalProp = (Boolean)value;
+                        } else if (propName.equals(UPDATE_CACHE_FREQUENCY)) {
+                            updateCacheFrequencyProp = (Long)value;
                         }
                     } 
                 }
@@ -2662,6 +2695,13 @@ public class MetaDataClient {
                         changingPhoenixTableProperty = true;
                     }
                 }
+                Long updateCacheFrequency = null;
+                if (updateCacheFrequencyProp != null) {
+                    if (updateCacheFrequencyProp.longValue() != table.getUpdateCacheFrequency()) {
+                        updateCacheFrequency = updateCacheFrequencyProp;
+                        changingPhoenixTableProperty = true;
+                    }
+                }
                 Boolean storeNulls = null;
                 if (storeNullsProp != null) {
                     if (storeNullsProp.booleanValue() != table.getStoreNulls()) {
@@ -2795,14 +2835,14 @@ public class MetaDataClient {
 
                 if (!table.getIndexes().isEmpty() && (numPkColumnsAdded>0 || nonTxToTx)) {
                     for (PTable index : table.getIndexes()) {
-                        incrementTableSeqNum(index, index.getType(), numPkColumnsAdded, nonTxToTx ? Boolean.TRUE : null);
+                        incrementTableSeqNum(index, index.getType(), numPkColumnsAdded, nonTxToTx ? Boolean.TRUE : null, updateCacheFrequency);
                     }
                     tableMetaData.addAll(connection.getMutationState().toMutations(timeStamp).next().getSecond());
                     connection.rollback();
                 }
                 long seqNum = table.getSequenceNumber();
                 if (changingPhoenixTableProperty || columnDefs.size() > 0) { 
-                    seqNum = incrementTableSeqNum(table, statement.getTableType(), columnDefs.size(), isTransactional, isImmutableRows, disableWAL, multiTenant, storeNulls);
+                    seqNum = incrementTableSeqNum(table, statement.getTableType(), columnDefs.size(), isTransactional, updateCacheFrequency, isImmutableRows, disableWAL, multiTenant, storeNulls);
                     tableMetaData.addAll(connection.getMutationState().toMutations(timeStamp).next().getSecond());
                     connection.rollback();
                 }
@@ -2844,19 +2884,26 @@ public class MetaDataClient {
                     // Only update client side cache if we aren't adding a PK column to a table with indexes or 
                     // transitioning a table from non transactional to transactional.
                     // We could update the cache manually then too, it'd just be a pain.
+                    String fullTableName = SchemaUtil.getTableName(schemaName, tableName);
+                    long resolvedTimeStamp = TransactionUtil.getResolvedTime(connection, result);
                     if (table.getIndexes().isEmpty() || (numPkColumnsAdded==0 && !nonTxToTx)) {
-                        connection.addColumn(
-                                tenantId,
-                                SchemaUtil.getTableName(schemaName, tableName),
-                                columns,
-                                result.getMutationTime(),
-                                seqNum,
-                                isImmutableRows == null ? table.isImmutableRows() : isImmutableRows,
-                                disableWAL == null ? table.isWALDisabled() : disableWAL,
-                                multiTenant == null ? table.isMultiTenant() : multiTenant,
-                                storeNulls == null ? table.getStoreNulls() : storeNulls, 
-                                isTransactional == null ? table.isTransactional() : isTransactional,
-                                TransactionUtil.getResolvedTime(connection, result));
+						connection.addColumn(
+								tenantId,
+								fullTableName,
+								columns,
+								result.getMutationTime(),
+								seqNum,
+								isImmutableRows == null ? table.isImmutableRows() : isImmutableRows,
+								disableWAL == null ? table.isWALDisabled() : disableWAL,
+								multiTenant == null ? table.isMultiTenant() : multiTenant,
+								storeNulls == null ? table.getStoreNulls() : storeNulls, 
+								isTransactional == null ? table.isTransactional() : isTransactional,
+								updateCacheFrequency == null ? table.getUpdateCacheFrequency() : updateCacheFrequency,
+								resolvedTimeStamp);
+                    } else if (updateCacheFrequency != null) {
+                        // Force removal from cache as the update cache frequency has changed
+                        // Note that clients outside this JVM won't be affected.
+                        connection.removeTable(tenantId, fullTableName, null, resolvedTimeStamp);
                     }
                     // Delete rows in view index if we haven't dropped it already
                     // We only need to do this if the multiTenant transitioned to false
@@ -3036,7 +3083,7 @@ public class MetaDataClient {
                         }
                     }
                     if(!indexColumnsToDrop.isEmpty()) {
-                        incrementTableSeqNum(index, index.getType(), -indexColumnsToDrop.size(), null);
+                        incrementTableSeqNum(index, index.getType(), -indexColumnsToDrop.size(), null, null);
                         dropColumnMutations(index, indexColumnsToDrop, tableMetaData);
                     }
 
@@ -3045,7 +3092,7 @@ public class MetaDataClient {
                 tableMetaData.addAll(connection.getMutationState().toMutations(timeStamp).next().getSecond());
                 connection.rollback();
 
-                long seqNum = incrementTableSeqNum(table, statement.getTableType(), -tableColumnsToDrop.size(), null);
+                long seqNum = incrementTableSeqNum(table, statement.getTableType(), -tableColumnsToDrop.size(), null, null);
                 tableMetaData.addAll(connection.getMutationState().toMutations(timeStamp).next().getSecond());
                 connection.rollback();
                 // Force table header to be first in list

http://git-wip-us.apache.org/repos/asf/phoenix/blob/59b336ec/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaData.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaData.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaData.java
index a3103bf..3adcb7e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaData.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaData.java
@@ -17,8 +17,6 @@
  */
 package org.apache.phoenix.schema;
 
-import java.sql.SQLException;
-
 import org.apache.phoenix.parse.PFunction;
 import org.apache.phoenix.query.MetaDataMutated;
 
@@ -34,4 +32,5 @@ public interface PMetaData extends MetaDataMutated, Iterable<PTable>, Cloneable
     public PMetaData pruneTables(Pruner pruner);
     public PFunction getFunction(PTableKey key) throws FunctionNotFoundException;
     public PMetaData pruneFunctions(Pruner pruner);
+    public long getAge(PTableRef ref);
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/59b336ec/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
index 9e4460d..66b4af3 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
@@ -170,6 +170,10 @@ public class PMetaDataImpl implements PMetaData {
                 return put(key, new PTableRef(value, timeKeeper.getCurrentTime(), 0, resolvedTime));
             }
             
+            public long getAge(PTableRef ref) {
+                return timeKeeper.getCurrentTime() - ref.getCreateTime();
+            }
+            
             public PTable remove(PTableKey key) {
                 PTableRef value = this.tables.remove(key);
                 if (value == null) {
@@ -313,7 +317,7 @@ public class PMetaDataImpl implements PMetaData {
     }
 
     @Override
-    public PMetaData addColumn(PName tenantId, String tableName, List<PColumn> columnsToAdd, long tableTimeStamp, long tableSeqNum, boolean isImmutableRows, boolean isWalDisabled, boolean isMultitenant, boolean storeNulls, boolean isTransactional, long resolvedTime) throws SQLException {
+    public PMetaData addColumn(PName tenantId, String tableName, List<PColumn> columnsToAdd, long tableTimeStamp, long tableSeqNum, boolean isImmutableRows, boolean isWalDisabled, boolean isMultitenant, boolean storeNulls, boolean isTransactional, long updateCacheFrequency, long resolvedTime) throws SQLException {
         PTableRef oldTableRef = metaData.get(new PTableKey(tenantId, tableName));
         if (oldTableRef == null) {
             return this;
@@ -327,7 +331,9 @@ public class PMetaDataImpl implements PMetaData {
             newColumns.addAll(oldColumns);
             newColumns.addAll(columnsToAdd);
         }
-        PTable newTable = PTableImpl.makePTable(oldTableRef.getTable(), tableTimeStamp, tableSeqNum, newColumns, isImmutableRows, isWalDisabled, isMultitenant, storeNulls, isTransactional);
+        PTable newTable = PTableImpl.makePTable(oldTableRef.getTable(),
+                tableTimeStamp, tableSeqNum, newColumns, isImmutableRows,
+                isWalDisabled, isMultitenant, storeNulls, isTransactional, updateCacheFrequency);
         return addTable(newTable, resolvedTime);
     }
 
@@ -472,4 +478,9 @@ public class PMetaDataImpl implements PMetaData {
         return new PMetaDataImpl(clone);
     
     }
+
+    @Override
+    public long getAge(PTableRef ref) {
+        return this.metaData.getAge(ref);
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/59b336ec/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
----------------------------------------------------------------------
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 ec97394..4a338f6 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
@@ -339,4 +339,5 @@ public interface PTable extends PMetaDataEntity {
      * -1 if there is no such column.
      */
     int getRowTimestampColPos();
+    long getUpdateCacheFrequency();
 }


[04/50] [abbrv] phoenix git commit: PHOENIX-2591 Minimize transaction commit/rollback for DDL

Posted by ma...@apache.org.
PHOENIX-2591 Minimize transaction commit/rollback for DDL


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

Branch: refs/heads/calcite
Commit: f591da44c9ee85ee7ab0fa910e3b18e649d86cdf
Parents: 9a44b49
Author: James Taylor <jt...@salesforce.com>
Authored: Mon Jan 18 21:09:44 2016 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Mon Jan 18 21:09:44 2016 -0800

----------------------------------------------------------------------
 .../apache/phoenix/execute/MutationState.java   | 30 ++++++++++----------
 .../apache/phoenix/util/TransactionUtil.java    | 23 ++++++---------
 .../java/org/apache/phoenix/util/TestUtil.java  |  2 +-
 3 files changed, 24 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/f591da44/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
index 35a36e6..ee694e7 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
@@ -37,18 +37,6 @@ import java.util.concurrent.TimeoutException;
 import javax.annotation.Nonnull;
 import javax.annotation.concurrent.Immutable;
 
-import co.cask.tephra.Transaction;
-import co.cask.tephra.Transaction.VisibilityLevel;
-import co.cask.tephra.TransactionAware;
-import co.cask.tephra.TransactionCodec;
-import co.cask.tephra.TransactionConflictException;
-import co.cask.tephra.TransactionContext;
-import co.cask.tephra.TransactionFailureException;
-import co.cask.tephra.TransactionSystemClient;
-import co.cask.tephra.hbase11.TransactionAwareHTable;
-import co.cask.tephra.visibility.FenceWait;
-import co.cask.tephra.visibility.VisibilityFence;
-
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.HTableInterface;
@@ -110,6 +98,18 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 
+import co.cask.tephra.Transaction;
+import co.cask.tephra.Transaction.VisibilityLevel;
+import co.cask.tephra.TransactionAware;
+import co.cask.tephra.TransactionCodec;
+import co.cask.tephra.TransactionConflictException;
+import co.cask.tephra.TransactionContext;
+import co.cask.tephra.TransactionFailureException;
+import co.cask.tephra.TransactionSystemClient;
+import co.cask.tephra.hbase11.TransactionAwareHTable;
+import co.cask.tephra.visibility.FenceWait;
+import co.cask.tephra.visibility.VisibilityFence;
+
 /**
  * 
  * Tracks the uncommitted state
@@ -369,9 +369,9 @@ public class MutationState implements SQLCloseable {
     	return getTransaction() != null;
     }
     
-    public long getReadPointer() {
-    	Transaction tx = getTransaction();
-    	return tx == null ? HConstants.LATEST_TIMESTAMP : tx.getReadPointer();
+    public long getInitialWritePointer() {
+        Transaction tx = getTransaction();
+        return tx == null ? HConstants.LATEST_TIMESTAMP : tx.getTransactionId(); // First write pointer - won't change with checkpointing
     }
     
     // For testing

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f591da44/phoenix-core/src/main/java/org/apache/phoenix/util/TransactionUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/TransactionUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/TransactionUtil.java
index 9f16e52..041c12e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/TransactionUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/TransactionUtil.java
@@ -19,6 +19,11 @@ package org.apache.phoenix.util;
 
 import java.sql.SQLException;
 
+import co.cask.tephra.TransactionConflictException;
+import co.cask.tephra.TransactionFailureException;
+import co.cask.tephra.TxConstants;
+import co.cask.tephra.hbase11.TransactionAwareHTable;
+
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.phoenix.coprocessor.MetaDataProtocol.MetaDataMutationResult;
 import org.apache.phoenix.exception.SQLExceptionCode;
@@ -27,11 +32,6 @@ import org.apache.phoenix.execute.MutationState;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.schema.PTable;
 
-import co.cask.tephra.TransactionConflictException;
-import co.cask.tephra.TransactionFailureException;
-import co.cask.tephra.TxConstants;
-import co.cask.tephra.hbase11.TransactionAwareHTable;
-
 public class TransactionUtil {
     private TransactionUtil() {
     }
@@ -67,7 +67,7 @@ public class TransactionUtil {
 	public static long getResolvedTimestamp(PhoenixConnection connection, boolean isTransactional, long defaultResolvedTimestamp) {
 		MutationState mutationState = connection.getMutationState();
 		Long scn = connection.getSCN();
-	    return scn != null ?  scn : (isTransactional && mutationState.isTransactionStarted()) ? convertToMilliseconds(mutationState.getReadPointer()) : defaultResolvedTimestamp;
+	    return scn != null ?  scn : (isTransactional && mutationState.isTransactionStarted()) ? convertToMilliseconds(mutationState.getInitialWritePointer()) : defaultResolvedTimestamp;
 	}
 
 	public static long getResolvedTime(PhoenixConnection connection, MetaDataMutationResult result) {
@@ -80,7 +80,7 @@ public class TransactionUtil {
 		PTable table = result.getTable();
 		MutationState mutationState = connection.getMutationState();
 		boolean txInProgress = table != null && table.isTransactional() && mutationState.isTransactionStarted();
-		return  txInProgress ? convertToMilliseconds(mutationState.getReadPointer()) : result.getMutationTime();
+		return  txInProgress ? convertToMilliseconds(mutationState.getInitialWritePointer()) : result.getMutationTime();
 	}
 
 	public static Long getTableTimestamp(PhoenixConnection connection, boolean transactional) throws SQLException {
@@ -89,17 +89,10 @@ public class TransactionUtil {
 			return timestamp;
 		}
 		MutationState mutationState = connection.getMutationState();
-		// we need to burn a txn so that we are sure the txn read pointer is close to wall clock time
 		if (!mutationState.isTransactionStarted()) {
 			mutationState.startTransaction();
-			connection.commit();
-		}
-		else {
-			connection.commit();
 		}
-		mutationState.startTransaction();
-		timestamp = convertToMilliseconds(mutationState.getReadPointer());
-		connection.commit();
+		timestamp = convertToMilliseconds(mutationState.getInitialWritePointer());
 		return timestamp;
 	}
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f591da44/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java b/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
index e1258e4..66a3c65 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
@@ -563,7 +563,7 @@ public class TestUtil {
         String query = "UPDATE STATISTICS " + tableName;
         conn.createStatement().execute(query);
         // if the table is transactional burn a txn in order to make sure the next txn read pointer is close to wall clock time
-        TransactionUtil.getTableTimestamp(conn.unwrap(PhoenixConnection.class), transactional);
+        conn.commit();
     }
     
     public static void analyzeTableIndex(Connection conn, String tableName) throws IOException, SQLException {


[31/50] [abbrv] phoenix git commit: PHOENIX-2626 System.Function is missing in the list of system tables in MetadataRpcController

Posted by ma...@apache.org.
PHOENIX-2626 System.Function is missing in the list of system tables in MetadataRpcController


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

Branch: refs/heads/calcite
Commit: 81b644f4ef2c2fdd51dd61cf3f544362cf4b2bd9
Parents: a6cd492
Author: Thomas D'Silva <td...@salesforce.com>
Authored: Fri Jan 22 16:35:21 2016 -0800
Committer: Thomas D'Silva <td...@salesforce.com>
Committed: Fri Jan 22 16:35:21 2016 -0800

----------------------------------------------------------------------
 .../apache/hadoop/hbase/ipc/controller/MetadataRpcController.java | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/81b644f4/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/controller/MetadataRpcController.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/controller/MetadataRpcController.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/controller/MetadataRpcController.java
index 23b9f03..2d51d08 100644
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/controller/MetadataRpcController.java
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/controller/MetadataRpcController.java
@@ -35,7 +35,8 @@ class MetadataRpcController extends DelegatingPayloadCarryingRpcController {
 	private static final List<String> SYSTEM_TABLE_NAMES = new ImmutableList.Builder<String>()
 			.add(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME)
 			.add(PhoenixDatabaseMetaData.SYSTEM_STATS_NAME)
-			.add(PhoenixDatabaseMetaData.SEQUENCE_FULLNAME).build();
+			.add(PhoenixDatabaseMetaData.SEQUENCE_FULLNAME)
+			.add(PhoenixDatabaseMetaData.SYSTEM_FUNCTION_NAME).build();
 
 	public MetadataRpcController(PayloadCarryingRpcController delegate,
 			Configuration conf) {


[16/50] [abbrv] phoenix git commit: PHOENIX-2615 Prevent builds from failing intermittenly due to timing out of forked jvms

Posted by ma...@apache.org.
PHOENIX-2615 Prevent builds from failing intermittenly due to timing out of forked jvms


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

Branch: refs/heads/calcite
Commit: 6ecbbb257cd09dbfe3f8ee7f3e6257660476c226
Parents: 6a27cec
Author: Samarth <sa...@salesforce.com>
Authored: Wed Jan 20 15:23:22 2016 -0800
Committer: Samarth <sa...@salesforce.com>
Committed: Wed Jan 20 15:23:22 2016 -0800

----------------------------------------------------------------------
 pom.xml | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/6ecbbb25/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 2c47ca2..7191b1e 100644
--- a/pom.xml
+++ b/pom.xml
@@ -273,6 +273,7 @@
                  <redirectTestOutputToFile>${test.output.tofile}</redirectTestOutputToFile>
                  <testSourceDirectory>${basedir}/src/it/java</testSourceDirectory>
                  <groups>org.apache.phoenix.end2end.ClientManagedTimeTest</groups>
+                 <shutdown>kill</shutdown>
                </configuration>
                <goals>
                  <goal>integration-test</goal>
@@ -290,6 +291,7 @@
                  <redirectTestOutputToFile>${test.output.tofile}</redirectTestOutputToFile>
                  <testSourceDirectory>${basedir}/src/it/java</testSourceDirectory>
                  <groups>org.apache.phoenix.end2end.HBaseManagedTimeTest</groups>
+                 <shutdown>kill</shutdown>
                </configuration>
                <goals>
                  <goal>integration-test</goal>
@@ -307,6 +309,7 @@
                  <redirectTestOutputToFile>${test.output.tofile}</redirectTestOutputToFile>
                  <testSourceDirectory>${basedir}/src/it/java</testSourceDirectory>
                  <groups>org.apache.phoenix.end2end.NeedsOwnMiniClusterTest</groups>
+                 <shutdown>kill</shutdown>
                </configuration>
                <goals>
                  <goal>integration-test</goal>


[27/50] [abbrv] phoenix git commit: PHOENIX-2621 ImmutableIndexIT is failing

Posted by ma...@apache.org.
PHOENIX-2621 ImmutableIndexIT is failing


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

Branch: refs/heads/calcite
Commit: 79d109e782b400af3006f5dc3dc810e3df4bd15e
Parents: a138cfe
Author: James Taylor <jt...@salesforce.com>
Authored: Fri Jan 22 00:58:46 2016 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Fri Jan 22 00:58:46 2016 -0800

----------------------------------------------------------------------
 .../phoenix/end2end/index/ImmutableIndexIT.java | 70 ++++++++++----------
 1 file changed, 36 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/79d109e7/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
index c18e4ab..ced3ac8 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
@@ -72,20 +72,19 @@ import com.google.common.collect.Maps;
 public class ImmutableIndexIT extends BaseHBaseManagedTimeIT {
 
     private final boolean localIndex;
-    private final boolean transactional;
     private final String tableDDLOptions;
     private final String tableName;
     private final String indexName;
     private final String fullTableName;
     private final String fullIndexName;
+    private volatile boolean stopThreads = false;
 
     private static String TABLE_NAME;
     private static String INDEX_DDL;
-    public static final AtomicInteger NUM_ROWS = new AtomicInteger(1);
+    public static final AtomicInteger NUM_ROWS = new AtomicInteger(0);
 
     public ImmutableIndexIT(boolean localIndex, boolean transactional) {
         this.localIndex = localIndex;
-        this.transactional = transactional;
         StringBuilder optionBuilder = new StringBuilder("IMMUTABLE_ROWS=true");
         if (transactional) {
             optionBuilder.append(", TRANSACTIONAL=true");
@@ -210,8 +209,8 @@ public class ImmutableIndexIT extends BaseHBaseManagedTimeIT {
         }
     }
 
-    private static class UpsertRunnable implements Runnable {
-        private static final int NUM_ROWS_IN_BATCH = 1000;
+    private class UpsertRunnable implements Runnable {
+        private static final int NUM_ROWS_IN_BATCH = 10;
         private final String fullTableName;
 
         public UpsertRunnable(String fullTableName) {
@@ -222,18 +221,19 @@ public class ImmutableIndexIT extends BaseHBaseManagedTimeIT {
         public void run() {
             Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
             try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-                while (true) {
+                while (!stopThreads) {
                     // write a large batch of rows
                     boolean fistRowInBatch = true;
-                    for (int i=0; i<NUM_ROWS_IN_BATCH; ++i) {
-                        BaseTest.upsertRow(conn, fullTableName, NUM_ROWS.intValue(), fistRowInBatch);
-                        NUM_ROWS.incrementAndGet();
+                    for (int i=0; i<NUM_ROWS_IN_BATCH && !stopThreads; ++i) {
+                        BaseTest.upsertRow(conn, fullTableName, NUM_ROWS.incrementAndGet(), fistRowInBatch);
                         fistRowInBatch = false;
                     }
                     conn.commit();
+                    Thread.sleep(10);
                 }
             } catch (SQLException e) {
                 throw new RuntimeException(e);
+            } catch (InterruptedException e) {
             }
         }
     }
@@ -242,56 +242,58 @@ public class ImmutableIndexIT extends BaseHBaseManagedTimeIT {
     public void testCreateIndexWhileUpsertingData() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
-        String indexDDL = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX IF NOT EXISTS " + indexName + " ON " + fullTableName
+        String indexDDL = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
                 + " (long_pk, varchar_pk)"
                 + " INCLUDE (long_col1, long_col2)";
-        int numThreads = 3;
+        int numThreads = 2;
+        ExecutorService executorService = Executors.newFixedThreadPool(numThreads, new ThreadFactory() {
+            @Override
+            public Thread newThread(Runnable r) {
+                Thread t = Executors.defaultThreadFactory().newThread(r);
+                t.setDaemon(true);
+                t.setPriority(Thread.MIN_PRIORITY);
+                return t;
+            }
+        });
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-            conn.setAutoCommit(false);
+            conn.setAutoCommit(true);
             Statement stmt = conn.createStatement();
             stmt.execute(ddl);
 
-            ExecutorService executorService = Executors.newFixedThreadPool(numThreads, new ThreadFactory() {
-                @Override
-                public Thread newThread(Runnable r) {
-                    Thread t = Executors.defaultThreadFactory().newThread(r);
-                    t.setDaemon(true);
-                    return t;
-                }
-            });
+            ResultSet rs;
+            rs = conn.createStatement().executeQuery("SELECT /*+ NO_INDEX */ COUNT(*) FROM " + fullTableName);
+            assertTrue(rs.next());
+            int dataTableRowCount = rs.getInt(1);
+            assertEquals(0,dataTableRowCount);
+
             List<Future<?>> futureList = Lists.newArrayListWithExpectedSize(numThreads);
             for (int i =0; i<numThreads; ++i) {
                 futureList.add(executorService.submit(new UpsertRunnable(fullTableName)));
             }
             // upsert some rows before creating the index 
-            Thread.sleep(500);
+            Thread.sleep(100);
 
             // create the index 
             try (Connection conn2 = DriverManager.getConnection(getUrl(), props)) {
-                conn2.setAutoCommit(false);
-                Statement stmt2 = conn2.createStatement();
-                stmt2.execute(indexDDL);
-                conn2.commit();
+                conn2.createStatement().execute(indexDDL);
             }
 
             // upsert some rows after creating the index
-            Thread.sleep(100);
+            Thread.sleep(50);
             // cancel the running threads
-            for (Future<?> future : futureList) {
-                future.cancel(true);
-            }
-            executorService.shutdownNow();
-            executorService.awaitTermination(30, TimeUnit.SECONDS);
-            Thread.sleep(100);
+            stopThreads = true;
+            executorService.shutdown();
+            assertTrue(executorService.awaitTermination(30, TimeUnit.SECONDS));
 
-            ResultSet rs;
             rs = conn.createStatement().executeQuery("SELECT /*+ NO_INDEX */ COUNT(*) FROM " + fullTableName);
             assertTrue(rs.next());
-            int dataTableRowCount = rs.getInt(1);
+            dataTableRowCount = rs.getInt(1);
             rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullIndexName);
             assertTrue(rs.next());
             int indexTableRowCount = rs.getInt(1);
             assertEquals("Data and Index table should have the same number of rows ", dataTableRowCount, indexTableRowCount);
+        } finally {
+            executorService.shutdownNow();
         }
     }
 


[10/50] [abbrv] phoenix git commit: PHOENIX-2303 Fix for performance.py not using JAVA_HOME (bpanneton)

Posted by ma...@apache.org.
PHOENIX-2303 Fix for performance.py not using JAVA_HOME (bpanneton)


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

Branch: refs/heads/calcite
Commit: 0b7a3c460e1171b5e67934684bf8ee71f07927cb
Parents: 15432d1
Author: Mujtaba <mu...@apache.org>
Authored: Mon Oct 5 10:57:35 2015 -0700
Committer: Nick Dimiduk <nd...@apache.org>
Committed: Tue Jan 19 20:49:52 2016 -0800

----------------------------------------------------------------------
 bin/performance.py | 36 +++++++++++++++++++++++++++++++++---
 1 file changed, 33 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/0b7a3c46/bin/performance.py
----------------------------------------------------------------------
diff --git a/bin/performance.py b/bin/performance.py
index 2f7fad1..fe39ddf 100755
--- a/bin/performance.py
+++ b/bin/performance.py
@@ -66,10 +66,40 @@ phoenix_utils.setPath()
 # HBase/Phoenix client side property override
 hbase_config_path = os.getenv('HBASE_CONF_DIR', phoenix_utils.current_dir)
 
-execute = ('java -cp "%s%s%s" -Dlog4j.configuration=file:' +
+java_home = os.getenv('JAVA_HOME')
+
+# load hbase-env.??? to extract JAVA_HOME, HBASE_PID_DIR, HBASE_LOG_DIR
+hbase_env_path = None
+hbase_env_cmd  = None
+if os.name == 'posix':
+    hbase_env_path = os.path.join(hbase_config_path, 'hbase-env.sh')
+    hbase_env_cmd = ['bash', '-c', 'source %s && env' % hbase_env_path]
+elif os.name == 'nt':
+    hbase_env_path = os.path.join(hbase_config_path, 'hbase-env.cmd')
+    hbase_env_cmd = ['cmd.exe', '/c', 'call %s & set' % hbase_env_path]
+if not hbase_env_path or not hbase_env_cmd:
+    print >> sys.stderr, "hbase-env file unknown on platform %s" % os.name
+    sys.exit(-1)
+
+hbase_env = {}
+if os.path.isfile(hbase_env_path):
+    p = subprocess.Popen(hbase_env_cmd, stdout = subprocess.PIPE)
+    for x in p.stdout:
+        (k, _, v) = x.partition('=')
+        hbase_env[k.strip()] = v.strip()
+
+if hbase_env.has_key('JAVA_HOME'):
+    java_home = hbase_env['JAVA_HOME']
+
+if java_home:
+    java_cmd = os.path.join(java_home, 'bin', 'java')
+else:
+    java_cmd = 'java'
+
+execute = ('%s -cp "%s%s%s" -Dlog4j.configuration=file:' +
            os.path.join(phoenix_utils.current_dir, "log4j.properties") +
            ' org.apache.phoenix.util.PhoenixRuntime -t %s %s ') % \
-    (hbase_config_path, os.pathsep, phoenix_utils.phoenix_client_jar, table, zookeeper)
+    (java_cmd, hbase_config_path, os.pathsep, phoenix_utils.phoenix_client_jar, table, zookeeper)
 
 # Create Table DDL
 createtable = "CREATE TABLE IF NOT EXISTS %s (HOST CHAR(2) NOT NULL,\
@@ -98,7 +128,7 @@ queryex("4 - Truncate + Group By", "SELECT TRUNC(DATE,'DAY') DAY FROM %s GROUP B
 queryex("5 - Filter + Count", "SELECT COUNT(1) FROM %s WHERE CORE<10;" % (table))
 
 print "\nGenerating and upserting data..."
-exitcode = subprocess.call('java -jar %s %s %s' % (phoenix_utils.testjar, data, rowcount),
+exitcode = subprocess.call('%s -jar %s %s %s' % (java_cmd, phoenix_utils.testjar, data, rowcount),
                            shell=True)
 if exitcode != 0:
     sys.exit(exitcode)


[46/50] [abbrv] phoenix git commit: PHOENIX-2543 Modify StatsCollectorIT to test transactional test cases

Posted by ma...@apache.org.
PHOENIX-2543 Modify StatsCollectorIT to test transactional test cases


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

Branch: refs/heads/calcite
Commit: 6af7dcb51e14d57f6a036fc718dc06f3f9e4ae41
Parents: 8a799b8
Author: Thomas D'Silva <td...@salesforce.com>
Authored: Thu Jan 28 13:26:40 2016 -0800
Committer: Thomas D'Silva <td...@salesforce.com>
Committed: Fri Jan 29 13:31:42 2016 -0800

----------------------------------------------------------------------
 .../phoenix/end2end/StatsCollectorIT.java       |  73 ++++++-----
 .../phoenix/end2end/TransactionalViewIT.java    | 120 +++++++++++++++++++
 .../org/apache/phoenix/tx/TransactionIT.java    |  39 +-----
 3 files changed, 169 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/6af7dcb5/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java
index 706ae4a..caba259 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java
@@ -30,6 +30,8 @@ import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
@@ -43,14 +45,23 @@ import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.TestUtil;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
 
 import com.google.common.collect.Maps;
 
-
+@RunWith(Parameterized.class)
 public class StatsCollectorIT extends StatsCollectorAbstractIT {
     private static final String STATS_TEST_TABLE_NAME = "S";
+    
+    private final String tableDDLOptions;
+    private final String tableName;
+    private final String fullTableName;
         
     @BeforeClass
     public static void doSetup() throws Exception {
@@ -62,30 +73,31 @@ public class StatsCollectorIT extends StatsCollectorAbstractIT {
         props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
         setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
     }
-
-    @Test
-    public void testUpdateStatsForNonTxnTable() throws Throwable {
-        helpTestUpdateStats(false);
+    
+    public StatsCollectorIT( boolean transactional) {
+        this.tableDDLOptions= transactional ? " TRANSACTIONAL=true" : "";
+        this.tableName = TestUtil.DEFAULT_DATA_TABLE_NAME + ( transactional ?  "_TXN" : "");
+        this.fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
     }
     
-    @Test
-    public void testUpdateStatsForTxnTable() throws Throwable {
-        helpTestUpdateStats(true);
+    @Parameters(name="transactional = {0}")
+    public static Collection<Boolean> data() {
+        return Arrays.asList(false,true);
     }
 
-	private void helpTestUpdateStats(boolean transactional) throws SQLException, IOException,
+    @Test
+    public void testUpdateStats() throws SQLException, IOException,
 			InterruptedException {
 		Connection conn;
         PreparedStatement stmt;
         ResultSet rs;
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        String tableName = "T" + (transactional ? "_TXN" : "");
         // props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 10));
         conn = DriverManager.getConnection(getUrl(), props);
         conn.createStatement().execute(
-                "CREATE TABLE " + tableName +" ( k VARCHAR, a_string_array VARCHAR(100) ARRAY[4], b_string_array VARCHAR(100) ARRAY[4] \n"
-                        + " CONSTRAINT pk PRIMARY KEY (k, b_string_array DESC))" 
-                		+ (transactional ? " TRANSACTIONAL=true" : ""));
+                "CREATE TABLE " + fullTableName +" ( k VARCHAR, a_string_array VARCHAR(100) ARRAY[4], b_string_array VARCHAR(100) ARRAY[4] \n"
+                        + " CONSTRAINT pk PRIMARY KEY (k, b_string_array DESC))"
+                		+ tableDDLOptions );
         String[] s;
         Array array;
         conn = upsertValues(props, tableName);
@@ -119,14 +131,14 @@ public class StatsCollectorIT extends StatsCollectorAbstractIT {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         conn = DriverManager.getConnection(getUrl(), props);
         conn.createStatement()
-                .execute("CREATE TABLE x ( k VARCHAR, c1.a bigint,c2.b bigint CONSTRAINT pk PRIMARY KEY (k)) \n");
-        conn.createStatement().execute("upsert into x values ('abc',1,3)");
-        conn.createStatement().execute("upsert into x values ('def',2,4)");
+                .execute("CREATE TABLE " + fullTableName +" ( k VARCHAR, c1.a bigint,c2.b bigint CONSTRAINT pk PRIMARY KEY (k))" + tableDDLOptions );
+        conn.createStatement().execute("upsert into " + fullTableName +" values ('abc',1,3)");
+        conn.createStatement().execute("upsert into " + fullTableName +" values ('def',2,4)");
         conn.commit();
         // CAll the update statistics query here
-        stmt = conn.prepareStatement("UPDATE STATISTICS X");
+        stmt = conn.prepareStatement("UPDATE STATISTICS " + fullTableName);
         stmt.execute();
-        rs = conn.createStatement().executeQuery("SELECT k FROM x");
+        rs = conn.createStatement().executeQuery("SELECT k FROM " + fullTableName);
         assertTrue(rs.next());
         assertEquals("abc", rs.getString(1));
         assertTrue(rs.next());
@@ -137,6 +149,7 @@ public class StatsCollectorIT extends StatsCollectorAbstractIT {
 
     @Test
     public void testUpdateStatsWithMultipleTables() throws Throwable {
+        String fullTableName2 = fullTableName+"_2";
         Connection conn;
         PreparedStatement stmt;
         ResultSet rs;
@@ -144,21 +157,21 @@ public class StatsCollectorIT extends StatsCollectorAbstractIT {
         // props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 10));
         conn = DriverManager.getConnection(getUrl(), props);
         conn.createStatement().execute(
-                "CREATE TABLE x ( k VARCHAR, a_string_array VARCHAR(100) ARRAY[4], b_string_array VARCHAR(100) ARRAY[4] \n"
-                        + " CONSTRAINT pk PRIMARY KEY (k, b_string_array DESC)) \n");
+                "CREATE TABLE " + fullTableName +" ( k VARCHAR, a_string_array VARCHAR(100) ARRAY[4], b_string_array VARCHAR(100) ARRAY[4] \n"
+                        + " CONSTRAINT pk PRIMARY KEY (k, b_string_array DESC))" + tableDDLOptions );
         conn.createStatement().execute(
-                "CREATE TABLE z ( k VARCHAR, a_string_array VARCHAR(100) ARRAY[4], b_string_array VARCHAR(100) ARRAY[4] \n"
-                        + " CONSTRAINT pk PRIMARY KEY (k, b_string_array DESC)) \n");
+                "CREATE TABLE " + fullTableName2 +" ( k VARCHAR, a_string_array VARCHAR(100) ARRAY[4], b_string_array VARCHAR(100) ARRAY[4] \n"
+                        + " CONSTRAINT pk PRIMARY KEY (k, b_string_array DESC))" + tableDDLOptions );
         String[] s;
         Array array;
-        conn = upsertValues(props, "x");
-        conn = upsertValues(props, "z");
+        conn = upsertValues(props, fullTableName);
+        conn = upsertValues(props, fullTableName2);
         // CAll the update statistics query here
-        stmt = conn.prepareStatement("UPDATE STATISTICS X");
+        stmt = conn.prepareStatement("UPDATE STATISTICS "+fullTableName);
         stmt.execute();
-        stmt = conn.prepareStatement("UPDATE STATISTICS Z");
+        stmt = conn.prepareStatement("UPDATE STATISTICS "+fullTableName2);
         stmt.execute();
-        stmt = upsertStmt(conn, "x");
+        stmt = upsertStmt(conn, fullTableName);
         stmt.setString(1, "z");
         s = new String[] { "xyz", "def", "ghi", "jkll", null, null, "xxx" };
         array = conn.createArrayOf("VARCHAR", s);
@@ -167,7 +180,7 @@ public class StatsCollectorIT extends StatsCollectorAbstractIT {
         array = conn.createArrayOf("VARCHAR", s);
         stmt.setArray(3, array);
         stmt.execute();
-        stmt = upsertStmt(conn, "z");
+        stmt = upsertStmt(conn, fullTableName2);
         stmt.setString(1, "z");
         s = new String[] { "xyz", "def", "ghi", "jkll", null, null, "xxx" };
         array = conn.createArrayOf("VARCHAR", s);
@@ -179,9 +192,9 @@ public class StatsCollectorIT extends StatsCollectorAbstractIT {
         conn.close();
         conn = DriverManager.getConnection(getUrl(), props);
         // This analyze would not work
-        stmt = conn.prepareStatement("UPDATE STATISTICS Z");
+        stmt = conn.prepareStatement("UPDATE STATISTICS "+fullTableName2);
         stmt.execute();
-        rs = conn.createStatement().executeQuery("SELECT k FROM Z");
+        rs = conn.createStatement().executeQuery("SELECT k FROM "+fullTableName2);
         assertTrue(rs.next());
         conn.close();
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6af7dcb5/phoenix-core/src/it/java/org/apache/phoenix/end2end/TransactionalViewIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TransactionalViewIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TransactionalViewIT.java
new file mode 100644
index 0000000..aa6fa06
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TransactionalViewIT.java
@@ -0,0 +1,120 @@
+/*
+ * 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.phoenix.end2end;
+
+import static org.apache.phoenix.util.TestUtil.analyzeTable;
+import static org.apache.phoenix.util.TestUtil.getAllSplits;
+import static org.junit.Assert.assertEquals;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.google.common.collect.Maps;
+
+public class TransactionalViewIT extends BaseOwnClusterHBaseManagedTimeIT {
+
+    @BeforeClass
+    public static void doSetup() throws Exception {
+        Map<String,String> props = Maps.newHashMapWithExpectedSize(3);
+        props.put(QueryServices.STATS_GUIDEPOST_WIDTH_BYTES_ATTRIB, Integer.toString(20));
+        props.put(QueryServices.QUEUE_SIZE_ATTRIB, Integer.toString(1024));
+        props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+    }
+    
+    @Test
+    public void testReadOwnWritesWithStats() throws Exception {
+        try (Connection conn1 = DriverManager.getConnection(getUrl()); 
+                Connection conn2 = DriverManager.getConnection(getUrl())) {
+            String ddl = "CREATE TABLE t (k INTEGER NOT NULL PRIMARY KEY, v1 DATE) TRANSACTIONAL=true";
+            conn1.createStatement().execute(ddl);
+            ddl = "CREATE VIEW v (v2 VARCHAR) AS SELECT * FROM t where k>5";
+            conn1.createStatement().execute(ddl);
+            for (int i = 0; i < 10; i++) {
+                conn1.createStatement().execute("UPSERT INTO t VALUES(" + i + ")");
+            }
+    
+            // verify you can read your own writes
+            int count = 0;
+            ResultSet rs = conn1.createStatement().executeQuery("SELECT k FROM t");
+            while (rs.next()) {
+                assertEquals(count++, rs.getInt(1));
+            }
+            assertEquals(10, count);
+            
+            count = 0;
+            rs = conn1.createStatement().executeQuery("SELECT k FROM v");
+            while (rs.next()) {
+                assertEquals(6+count++, rs.getInt(1));
+            }
+            assertEquals(4, count);
+            
+            // verify stats can see the read own writes rows
+            analyzeTable(conn2, "v", true);
+            List<KeyRange> splits = getAllSplits(conn2, "v");
+            assertEquals(4, splits.size());
+        }
+    }
+    
+    @Test
+    public void testInvalidRowsWithStats() throws Exception {
+        try (Connection conn1 = DriverManager.getConnection(getUrl()); 
+                Connection conn2 = DriverManager.getConnection(getUrl())) {
+            String ddl = "CREATE TABLE t (k INTEGER NOT NULL PRIMARY KEY, v1 DATE) TRANSACTIONAL=true";
+            conn1.createStatement().execute(ddl);
+            ddl = "CREATE VIEW v (v2 VARCHAR) AS SELECT * FROM t where k>5";
+            conn1.createStatement().execute(ddl);
+            for (int i = 0; i < 10; i++) {
+                conn1.createStatement().execute("UPSERT INTO t VALUES(" + i + ")");
+            }
+    
+            // verify you can read your own writes
+            int count = 0;
+            ResultSet rs = conn1.createStatement().executeQuery("SELECT k FROM t");
+            while (rs.next()) {
+                assertEquals(count++, rs.getInt(1));
+            }
+            assertEquals(10, count);
+            
+            count = 0;
+            rs = conn1.createStatement().executeQuery("SELECT k FROM v");
+            while (rs.next()) {
+                assertEquals(6+count++, rs.getInt(1));
+            }
+            assertEquals(4, count);
+            
+            Thread.sleep(DEFAULT_TXN_TIMEOUT_SECONDS*1000+20000);
+            assertEquals("There should be one invalid transaction", 1, txManager.getInvalidSize());
+            
+            // verify stats can see the rows from the invalid transaction
+            analyzeTable(conn2, "v", true);
+            List<KeyRange> splits = getAllSplits(conn2, "v");
+            assertEquals(4, splits.size());
+        }
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6af7dcb5/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
index 2794c47..1bf313b 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
@@ -36,12 +36,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 
-import co.cask.tephra.TransactionContext;
-import co.cask.tephra.TransactionSystemClient;
-import co.cask.tephra.TxConstants;
-import co.cask.tephra.hbase11.TransactionAwareHTable;
-import co.cask.tephra.hbase11.coprocessor.TransactionProcessor;
-
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
@@ -71,6 +65,12 @@ import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
 
+import co.cask.tephra.TransactionContext;
+import co.cask.tephra.TransactionSystemClient;
+import co.cask.tephra.TxConstants;
+import co.cask.tephra.hbase11.TransactionAwareHTable;
+import co.cask.tephra.hbase11.coprocessor.TransactionProcessor;
+
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
@@ -581,33 +581,6 @@ public class TransactionIT extends BaseHBaseManagedTimeIT {
     }
     
     @Test
-    public void testReadOnlyView() throws Exception {
-        Connection conn = DriverManager.getConnection(getUrl());
-        String ddl = "CREATE TABLE t (k INTEGER NOT NULL PRIMARY KEY, v1 DATE) TRANSACTIONAL=true";
-        conn.createStatement().execute(ddl);
-        ddl = "CREATE VIEW v (v2 VARCHAR) AS SELECT * FROM t where k>4";
-        conn.createStatement().execute(ddl);
-        for (int i = 0; i < 10; i++) {
-            conn.createStatement().execute("UPSERT INTO t VALUES(" + i + ")");
-        }
-        conn.commit();
-        
-        int count = 0;
-        ResultSet rs = conn.createStatement().executeQuery("SELECT k FROM t");
-        while (rs.next()) {
-            assertEquals(count++, rs.getInt(1));
-        }
-        assertEquals(10, count);
-        
-        count = 0;
-        rs = conn.createStatement().executeQuery("SELECT k FROM v");
-        while (rs.next()) {
-            assertEquals(5+count++, rs.getInt(1));
-        }
-        assertEquals(5, count);
-    }
-    
-    @Test
     public void testExternalTxContext() throws Exception {
         ResultSet rs;
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);


[15/50] [abbrv] phoenix git commit: PHOENIX-2417 Compress memory used by row key byte[] of guideposts (Ankit Singhal)

Posted by ma...@apache.org.
PHOENIX-2417 Compress memory used by row key byte[] of guideposts (Ankit Singhal)


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

Branch: refs/heads/calcite
Commit: 6a27cecce312002f9b7f76ffdcafcb694f5e3ea8
Parents: 4d323b6
Author: James Taylor <jt...@salesforce.com>
Authored: Wed Jan 20 09:57:14 2016 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Wed Jan 20 10:20:45 2016 -0800

----------------------------------------------------------------------
 .../coprocessor/MetaDataRegionObserver.java     |   71 +-
 .../generated/PGuidePostsProtos.java            |  287 +++-
 .../generated/StatCollectorProtos.java          | 1269 ------------------
 .../org/apache/phoenix/execute/ScanPlan.java    |    2 +-
 .../phoenix/iterate/BaseResultIterators.java    |  138 +-
 .../org/apache/phoenix/schema/PTableImpl.java   |   39 +-
 .../phoenix/schema/stats/GuidePostsInfo.java    |  106 +-
 .../schema/stats/GuidePostsInfoBuilder.java     |  113 ++
 .../phoenix/schema/stats/PTableStatsImpl.java   |   45 +-
 .../schema/stats/StatisticsCollector.java       |   46 +-
 .../phoenix/schema/stats/StatisticsScanner.java |    3 +-
 .../phoenix/schema/stats/StatisticsUtil.java    |   36 +-
 .../phoenix/schema/stats/StatisticsWriter.java  |   46 +-
 .../java/org/apache/phoenix/util/ByteUtil.java  |    2 +
 .../apache/phoenix/util/PrefixByteCodec.java    |  104 ++
 .../apache/phoenix/util/PrefixByteDecoder.java  |   90 ++
 .../apache/phoenix/util/PrefixByteEncoder.java  |   99 ++
 .../org/apache/phoenix/util/UpgradeUtil.java    |   68 +-
 .../phoenix/filter/SkipScanBigFilterTest.java   |    8 +-
 .../util/PrefixByteEncoderDecoderTest.java      |   96 ++
 phoenix-protocol/src/main/PGuidePosts.proto     |    3 +
 21 files changed, 1140 insertions(+), 1531 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/6a27cecc/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
index c40e3cd..7950ac8 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
@@ -18,10 +18,6 @@
 package org.apache.phoenix.coprocessor;
 
 import java.io.IOException;
-
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-import org.apache.phoenix.jdbc.PhoenixDriver;
 import java.sql.DriverManager;
 import java.sql.SQLException;
 import java.util.ArrayList;
@@ -36,28 +32,34 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
 import org.apache.hadoop.hbase.filter.CompareFilter;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
 import org.apache.phoenix.cache.GlobalCache;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.jdbc.PhoenixDriver;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.schema.MetaDataClient;
 import org.apache.phoenix.schema.PIndexState;
-import org.apache.phoenix.schema.types.PLong;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.schema.types.PLong;
 import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.UpgradeUtil;
 
 
 /**
@@ -96,10 +98,52 @@ public class MetaDataRegionObserver extends BaseRegionObserver {
         rebuildIndexTimeInterval = env.getConfiguration().getLong(QueryServices.INDEX_FAILURE_HANDLING_REBUILD_INTERVAL_ATTRIB,
             QueryServicesOptions.DEFAULT_INDEX_FAILURE_HANDLING_REBUILD_INTERVAL);
     }
-
+    
+    private static String getJdbcUrl(RegionCoprocessorEnvironment env) {
+        String zkQuorum = env.getConfiguration().get(HConstants.ZOOKEEPER_QUORUM);
+        String zkClientPort = env.getConfiguration().get(HConstants.ZOOKEEPER_CLIENT_PORT,
+            Integer.toString(HConstants.DEFAULT_ZOOKEPER_CLIENT_PORT));
+        String zkParentNode = env.getConfiguration().get(HConstants.ZOOKEEPER_ZNODE_PARENT,
+            HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
+        return PhoenixRuntime.JDBC_PROTOCOL + PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR + zkQuorum
+            + PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR + zkClientPort
+            + PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR + zkParentNode;
+    }
 
     @Override
     public void postOpen(ObserverContext<RegionCoprocessorEnvironment> e) {
+        final RegionCoprocessorEnvironment env = e.getEnvironment();
+
+        Runnable r = new Runnable() {
+            @Override
+            public void run() {
+                HTableInterface metaTable = null;
+                HTableInterface statsTable = null;
+                try {
+                    Thread.sleep(1000);
+                    metaTable = env.getTable(TableName.valueOf(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME));
+                    statsTable = env.getTable(TableName.valueOf(PhoenixDatabaseMetaData.SYSTEM_STATS_NAME));
+                    if (UpgradeUtil.truncateStats(metaTable, statsTable)) {
+                        LOG.info("Stats are successfully truncated for upgrade 4.7!!");
+                    }
+                } catch (Exception exception) {
+                    LOG.warn("Exception while truncate stats..,"
+                            + " please check and delete stats manually inorder to get proper result with old client!!");
+                    LOG.warn(exception.getStackTrace());
+                } finally {
+                    try {
+                        if (metaTable != null) {
+                            metaTable.close();
+                        }
+                        if (statsTable != null) {
+                            statsTable.close();
+                        }
+                    } catch (IOException e) {}
+                }
+            }
+        };
+        (new Thread(r)).start();
+
         if (!enableRebuildIndex) {
             LOG.info("Failure Index Rebuild is skipped by configuration.");
             return;
@@ -134,17 +178,6 @@ public class MetaDataRegionObserver extends BaseRegionObserver {
             this.env = env;
         }
 
-        private String getJdbcUrl() {
-            String zkQuorum = this.env.getConfiguration().get(HConstants.ZOOKEEPER_QUORUM);
-            String zkClientPort = this.env.getConfiguration().get(HConstants.ZOOKEEPER_CLIENT_PORT,
-                Integer.toString(HConstants.DEFAULT_ZOOKEPER_CLIENT_PORT));
-            String zkParentNode = this.env.getConfiguration().get(HConstants.ZOOKEEPER_ZNODE_PARENT,
-                HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
-            return PhoenixRuntime.JDBC_PROTOCOL + PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR + zkQuorum
-                + PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR + zkClientPort
-                + PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR + zkParentNode;
-        }
-
         @Override
         public void run() {
             RegionScanner scanner = null;
@@ -219,7 +252,7 @@ public class MetaDataRegionObserver extends BaseRegionObserver {
                     }
 
                     if (conn == null) {
-                        conn = DriverManager.getConnection(getJdbcUrl()).unwrap(PhoenixConnection.class);
+                        conn = DriverManager.getConnection(getJdbcUrl(env)).unwrap(PhoenixConnection.class);
                     }
 
                     String dataTableFullName = SchemaUtil.getTableName(schemaName, dataTable);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6a27cecc/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PGuidePostsProtos.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PGuidePostsProtos.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PGuidePostsProtos.java
index 9f96530..d9dcb56 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PGuidePostsProtos.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PGuidePostsProtos.java
@@ -44,6 +44,36 @@ public final class PGuidePostsProtos {
      * <code>optional int64 rowCount = 3;</code>
      */
     long getRowCount();
+
+    // optional int32 maxLength = 4;
+    /**
+     * <code>optional int32 maxLength = 4;</code>
+     */
+    boolean hasMaxLength();
+    /**
+     * <code>optional int32 maxLength = 4;</code>
+     */
+    int getMaxLength();
+
+    // optional int32 encodedGuidePostsCount = 5;
+    /**
+     * <code>optional int32 encodedGuidePostsCount = 5;</code>
+     */
+    boolean hasEncodedGuidePostsCount();
+    /**
+     * <code>optional int32 encodedGuidePostsCount = 5;</code>
+     */
+    int getEncodedGuidePostsCount();
+
+    // optional bytes encodedGuidePosts = 6;
+    /**
+     * <code>optional bytes encodedGuidePosts = 6;</code>
+     */
+    boolean hasEncodedGuidePosts();
+    /**
+     * <code>optional bytes encodedGuidePosts = 6;</code>
+     */
+    com.google.protobuf.ByteString getEncodedGuidePosts();
   }
   /**
    * Protobuf type {@code PGuidePosts}
@@ -114,6 +144,21 @@ public final class PGuidePostsProtos {
               rowCount_ = input.readInt64();
               break;
             }
+            case 32: {
+              bitField0_ |= 0x00000004;
+              maxLength_ = input.readInt32();
+              break;
+            }
+            case 40: {
+              bitField0_ |= 0x00000008;
+              encodedGuidePostsCount_ = input.readInt32();
+              break;
+            }
+            case 50: {
+              bitField0_ |= 0x00000010;
+              encodedGuidePosts_ = input.readBytes();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -212,10 +257,61 @@ public final class PGuidePostsProtos {
       return rowCount_;
     }
 
+    // optional int32 maxLength = 4;
+    public static final int MAXLENGTH_FIELD_NUMBER = 4;
+    private int maxLength_;
+    /**
+     * <code>optional int32 maxLength = 4;</code>
+     */
+    public boolean hasMaxLength() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>optional int32 maxLength = 4;</code>
+     */
+    public int getMaxLength() {
+      return maxLength_;
+    }
+
+    // optional int32 encodedGuidePostsCount = 5;
+    public static final int ENCODEDGUIDEPOSTSCOUNT_FIELD_NUMBER = 5;
+    private int encodedGuidePostsCount_;
+    /**
+     * <code>optional int32 encodedGuidePostsCount = 5;</code>
+     */
+    public boolean hasEncodedGuidePostsCount() {
+      return ((bitField0_ & 0x00000008) == 0x00000008);
+    }
+    /**
+     * <code>optional int32 encodedGuidePostsCount = 5;</code>
+     */
+    public int getEncodedGuidePostsCount() {
+      return encodedGuidePostsCount_;
+    }
+
+    // optional bytes encodedGuidePosts = 6;
+    public static final int ENCODEDGUIDEPOSTS_FIELD_NUMBER = 6;
+    private com.google.protobuf.ByteString encodedGuidePosts_;
+    /**
+     * <code>optional bytes encodedGuidePosts = 6;</code>
+     */
+    public boolean hasEncodedGuidePosts() {
+      return ((bitField0_ & 0x00000010) == 0x00000010);
+    }
+    /**
+     * <code>optional bytes encodedGuidePosts = 6;</code>
+     */
+    public com.google.protobuf.ByteString getEncodedGuidePosts() {
+      return encodedGuidePosts_;
+    }
+
     private void initFields() {
       guidePosts_ = java.util.Collections.emptyList();
       byteCount_ = 0L;
       rowCount_ = 0L;
+      maxLength_ = 0;
+      encodedGuidePostsCount_ = 0;
+      encodedGuidePosts_ = com.google.protobuf.ByteString.EMPTY;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -238,6 +334,15 @@ public final class PGuidePostsProtos {
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         output.writeInt64(3, rowCount_);
       }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeInt32(4, maxLength_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        output.writeInt32(5, encodedGuidePostsCount_);
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        output.writeBytes(6, encodedGuidePosts_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -264,6 +369,18 @@ public final class PGuidePostsProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeInt64Size(3, rowCount_);
       }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt32Size(4, maxLength_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt32Size(5, encodedGuidePostsCount_);
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(6, encodedGuidePosts_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -299,6 +416,21 @@ public final class PGuidePostsProtos {
         result = result && (getRowCount()
             == other.getRowCount());
       }
+      result = result && (hasMaxLength() == other.hasMaxLength());
+      if (hasMaxLength()) {
+        result = result && (getMaxLength()
+            == other.getMaxLength());
+      }
+      result = result && (hasEncodedGuidePostsCount() == other.hasEncodedGuidePostsCount());
+      if (hasEncodedGuidePostsCount()) {
+        result = result && (getEncodedGuidePostsCount()
+            == other.getEncodedGuidePostsCount());
+      }
+      result = result && (hasEncodedGuidePosts() == other.hasEncodedGuidePosts());
+      if (hasEncodedGuidePosts()) {
+        result = result && getEncodedGuidePosts()
+            .equals(other.getEncodedGuidePosts());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -324,6 +456,18 @@ public final class PGuidePostsProtos {
         hash = (37 * hash) + ROWCOUNT_FIELD_NUMBER;
         hash = (53 * hash) + hashLong(getRowCount());
       }
+      if (hasMaxLength()) {
+        hash = (37 * hash) + MAXLENGTH_FIELD_NUMBER;
+        hash = (53 * hash) + getMaxLength();
+      }
+      if (hasEncodedGuidePostsCount()) {
+        hash = (37 * hash) + ENCODEDGUIDEPOSTSCOUNT_FIELD_NUMBER;
+        hash = (53 * hash) + getEncodedGuidePostsCount();
+      }
+      if (hasEncodedGuidePosts()) {
+        hash = (37 * hash) + ENCODEDGUIDEPOSTS_FIELD_NUMBER;
+        hash = (53 * hash) + getEncodedGuidePosts().hashCode();
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -439,6 +583,12 @@ public final class PGuidePostsProtos {
         bitField0_ = (bitField0_ & ~0x00000002);
         rowCount_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000004);
+        maxLength_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000008);
+        encodedGuidePostsCount_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000010);
+        encodedGuidePosts_ = com.google.protobuf.ByteString.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000020);
         return this;
       }
 
@@ -480,6 +630,18 @@ public final class PGuidePostsProtos {
           to_bitField0_ |= 0x00000002;
         }
         result.rowCount_ = rowCount_;
+        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.maxLength_ = maxLength_;
+        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
+          to_bitField0_ |= 0x00000008;
+        }
+        result.encodedGuidePostsCount_ = encodedGuidePostsCount_;
+        if (((from_bitField0_ & 0x00000020) == 0x00000020)) {
+          to_bitField0_ |= 0x00000010;
+        }
+        result.encodedGuidePosts_ = encodedGuidePosts_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -512,6 +674,15 @@ public final class PGuidePostsProtos {
         if (other.hasRowCount()) {
           setRowCount(other.getRowCount());
         }
+        if (other.hasMaxLength()) {
+          setMaxLength(other.getMaxLength());
+        }
+        if (other.hasEncodedGuidePostsCount()) {
+          setEncodedGuidePostsCount(other.getEncodedGuidePostsCount());
+        }
+        if (other.hasEncodedGuidePosts()) {
+          setEncodedGuidePosts(other.getEncodedGuidePosts());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -677,6 +848,108 @@ public final class PGuidePostsProtos {
         return this;
       }
 
+      // optional int32 maxLength = 4;
+      private int maxLength_ ;
+      /**
+       * <code>optional int32 maxLength = 4;</code>
+       */
+      public boolean hasMaxLength() {
+        return ((bitField0_ & 0x00000008) == 0x00000008);
+      }
+      /**
+       * <code>optional int32 maxLength = 4;</code>
+       */
+      public int getMaxLength() {
+        return maxLength_;
+      }
+      /**
+       * <code>optional int32 maxLength = 4;</code>
+       */
+      public Builder setMaxLength(int value) {
+        bitField0_ |= 0x00000008;
+        maxLength_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional int32 maxLength = 4;</code>
+       */
+      public Builder clearMaxLength() {
+        bitField0_ = (bitField0_ & ~0x00000008);
+        maxLength_ = 0;
+        onChanged();
+        return this;
+      }
+
+      // optional int32 encodedGuidePostsCount = 5;
+      private int encodedGuidePostsCount_ ;
+      /**
+       * <code>optional int32 encodedGuidePostsCount = 5;</code>
+       */
+      public boolean hasEncodedGuidePostsCount() {
+        return ((bitField0_ & 0x00000010) == 0x00000010);
+      }
+      /**
+       * <code>optional int32 encodedGuidePostsCount = 5;</code>
+       */
+      public int getEncodedGuidePostsCount() {
+        return encodedGuidePostsCount_;
+      }
+      /**
+       * <code>optional int32 encodedGuidePostsCount = 5;</code>
+       */
+      public Builder setEncodedGuidePostsCount(int value) {
+        bitField0_ |= 0x00000010;
+        encodedGuidePostsCount_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional int32 encodedGuidePostsCount = 5;</code>
+       */
+      public Builder clearEncodedGuidePostsCount() {
+        bitField0_ = (bitField0_ & ~0x00000010);
+        encodedGuidePostsCount_ = 0;
+        onChanged();
+        return this;
+      }
+
+      // optional bytes encodedGuidePosts = 6;
+      private com.google.protobuf.ByteString encodedGuidePosts_ = com.google.protobuf.ByteString.EMPTY;
+      /**
+       * <code>optional bytes encodedGuidePosts = 6;</code>
+       */
+      public boolean hasEncodedGuidePosts() {
+        return ((bitField0_ & 0x00000020) == 0x00000020);
+      }
+      /**
+       * <code>optional bytes encodedGuidePosts = 6;</code>
+       */
+      public com.google.protobuf.ByteString getEncodedGuidePosts() {
+        return encodedGuidePosts_;
+      }
+      /**
+       * <code>optional bytes encodedGuidePosts = 6;</code>
+       */
+      public Builder setEncodedGuidePosts(com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000020;
+        encodedGuidePosts_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bytes encodedGuidePosts = 6;</code>
+       */
+      public Builder clearEncodedGuidePosts() {
+        bitField0_ = (bitField0_ & ~0x00000020);
+        encodedGuidePosts_ = getDefaultInstance().getEncodedGuidePosts();
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:PGuidePosts)
     }
 
@@ -702,11 +975,13 @@ public final class PGuidePostsProtos {
       descriptor;
   static {
     java.lang.String[] descriptorData = {
-      "\n\021PGuidePosts.proto\"F\n\013PGuidePosts\022\022\n\ngu" +
-      "idePosts\030\001 \003(\014\022\021\n\tbyteCount\030\002 \001(\003\022\020\n\010row" +
-      "Count\030\003 \001(\003BE\n(org.apache.phoenix.coproc" +
-      "essor.generatedB\021PGuidePostsProtosH\001\210\001\001\240" +
-      "\001\001"
+      "\n\021PGuidePosts.proto\"\224\001\n\013PGuidePosts\022\022\n\ng" +
+      "uidePosts\030\001 \003(\014\022\021\n\tbyteCount\030\002 \001(\003\022\020\n\010ro" +
+      "wCount\030\003 \001(\003\022\021\n\tmaxLength\030\004 \001(\005\022\036\n\026encod" +
+      "edGuidePostsCount\030\005 \001(\005\022\031\n\021encodedGuideP" +
+      "osts\030\006 \001(\014BE\n(org.apache.phoenix.coproce" +
+      "ssor.generatedB\021PGuidePostsProtosH\001\210\001\001\240\001" +
+      "\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -718,7 +993,7 @@ public final class PGuidePostsProtos {
           internal_static_PGuidePosts_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_PGuidePosts_descriptor,
-              new java.lang.String[] { "GuidePosts", "ByteCount", "RowCount", });
+              new java.lang.String[] { "GuidePosts", "ByteCount", "RowCount", "MaxLength", "EncodedGuidePostsCount", "EncodedGuidePosts", });
           return null;
         }
       };

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6a27cecc/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/StatCollectorProtos.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/StatCollectorProtos.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/StatCollectorProtos.java
deleted file mode 100644
index 5008f22..0000000
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/StatCollectorProtos.java
+++ /dev/null
@@ -1,1269 +0,0 @@
-// Generated by the protocol buffer compiler.  DO NOT EDIT!
-// source: StatisticsCollect.proto
-
-package org.apache.phoenix.coprocessor.generated;
-
-public final class StatCollectorProtos {
-  private StatCollectorProtos() {}
-  public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistry registry) {
-  }
-  public interface StatCollectRequestOrBuilder
-      extends com.google.protobuf.MessageOrBuilder {
-
-    // optional bytes startRow = 1;
-    /**
-     * <code>optional bytes startRow = 1;</code>
-     */
-    boolean hasStartRow();
-    /**
-     * <code>optional bytes startRow = 1;</code>
-     */
-    com.google.protobuf.ByteString getStartRow();
-
-    // optional bytes stopRow = 2;
-    /**
-     * <code>optional bytes stopRow = 2;</code>
-     */
-    boolean hasStopRow();
-    /**
-     * <code>optional bytes stopRow = 2;</code>
-     */
-    com.google.protobuf.ByteString getStopRow();
-  }
-  /**
-   * Protobuf type {@code StatCollectRequest}
-   */
-  public static final class StatCollectRequest extends
-      com.google.protobuf.GeneratedMessage
-      implements StatCollectRequestOrBuilder {
-    // Use StatCollectRequest.newBuilder() to construct.
-    private StatCollectRequest(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
-      super(builder);
-      this.unknownFields = builder.getUnknownFields();
-    }
-    private StatCollectRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
-
-    private static final StatCollectRequest defaultInstance;
-    public static StatCollectRequest getDefaultInstance() {
-      return defaultInstance;
-    }
-
-    public StatCollectRequest getDefaultInstanceForType() {
-      return defaultInstance;
-    }
-
-    private final com.google.protobuf.UnknownFieldSet unknownFields;
-    @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
-        getUnknownFields() {
-      return this.unknownFields;
-    }
-    private StatCollectRequest(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      initFields();
-      int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
-      try {
-        boolean done = false;
-        while (!done) {
-          int tag = input.readTag();
-          switch (tag) {
-            case 0:
-              done = true;
-              break;
-            default: {
-              if (!parseUnknownField(input, unknownFields,
-                                     extensionRegistry, tag)) {
-                done = true;
-              }
-              break;
-            }
-            case 10: {
-              bitField0_ |= 0x00000001;
-              startRow_ = input.readBytes();
-              break;
-            }
-            case 18: {
-              bitField0_ |= 0x00000002;
-              stopRow_ = input.readBytes();
-              break;
-            }
-          }
-        }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-        throw e.setUnfinishedMessage(this);
-      } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
-            e.getMessage()).setUnfinishedMessage(this);
-      } finally {
-        this.unknownFields = unknownFields.build();
-        makeExtensionsImmutable();
-      }
-    }
-    public static final com.google.protobuf.Descriptors.Descriptor
-        getDescriptor() {
-      return org.apache.phoenix.coprocessor.generated.StatCollectorProtos.internal_static_StatCollectRequest_descriptor;
-    }
-
-    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
-        internalGetFieldAccessorTable() {
-      return org.apache.phoenix.coprocessor.generated.StatCollectorProtos.internal_static_StatCollectRequest_fieldAccessorTable
-          .ensureFieldAccessorsInitialized(
-              org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest.class, org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest.Builder.class);
-    }
-
-    public static com.google.protobuf.Parser<StatCollectRequest> PARSER =
-        new com.google.protobuf.AbstractParser<StatCollectRequest>() {
-      public StatCollectRequest parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
-        return new StatCollectRequest(input, extensionRegistry);
-      }
-    };
-
-    @java.lang.Override
-    public com.google.protobuf.Parser<StatCollectRequest> getParserForType() {
-      return PARSER;
-    }
-
-    private int bitField0_;
-    // optional bytes startRow = 1;
-    public static final int STARTROW_FIELD_NUMBER = 1;
-    private com.google.protobuf.ByteString startRow_;
-    /**
-     * <code>optional bytes startRow = 1;</code>
-     */
-    public boolean hasStartRow() {
-      return ((bitField0_ & 0x00000001) == 0x00000001);
-    }
-    /**
-     * <code>optional bytes startRow = 1;</code>
-     */
-    public com.google.protobuf.ByteString getStartRow() {
-      return startRow_;
-    }
-
-    // optional bytes stopRow = 2;
-    public static final int STOPROW_FIELD_NUMBER = 2;
-    private com.google.protobuf.ByteString stopRow_;
-    /**
-     * <code>optional bytes stopRow = 2;</code>
-     */
-    public boolean hasStopRow() {
-      return ((bitField0_ & 0x00000002) == 0x00000002);
-    }
-    /**
-     * <code>optional bytes stopRow = 2;</code>
-     */
-    public com.google.protobuf.ByteString getStopRow() {
-      return stopRow_;
-    }
-
-    private void initFields() {
-      startRow_ = com.google.protobuf.ByteString.EMPTY;
-      stopRow_ = com.google.protobuf.ByteString.EMPTY;
-    }
-    private byte memoizedIsInitialized = -1;
-    public final boolean isInitialized() {
-      byte isInitialized = memoizedIsInitialized;
-      if (isInitialized != -1) return isInitialized == 1;
-
-      memoizedIsInitialized = 1;
-      return true;
-    }
-
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
-                        throws java.io.IOException {
-      getSerializedSize();
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        output.writeBytes(1, startRow_);
-      }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        output.writeBytes(2, stopRow_);
-      }
-      getUnknownFields().writeTo(output);
-    }
-
-    private int memoizedSerializedSize = -1;
-    public int getSerializedSize() {
-      int size = memoizedSerializedSize;
-      if (size != -1) return size;
-
-      size = 0;
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(1, startRow_);
-      }
-      if (((bitField0_ & 0x00000002) == 0x00000002)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBytesSize(2, stopRow_);
-      }
-      size += getUnknownFields().getSerializedSize();
-      memoizedSerializedSize = size;
-      return size;
-    }
-
-    private static final long serialVersionUID = 0L;
-    @java.lang.Override
-    protected java.lang.Object writeReplace()
-        throws java.io.ObjectStreamException {
-      return super.writeReplace();
-    }
-
-    @java.lang.Override
-    public boolean equals(final java.lang.Object obj) {
-      if (obj == this) {
-       return true;
-      }
-      if (!(obj instanceof org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest)) {
-        return super.equals(obj);
-      }
-      org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest other = (org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest) obj;
-
-      boolean result = true;
-      result = result && (hasStartRow() == other.hasStartRow());
-      if (hasStartRow()) {
-        result = result && getStartRow()
-            .equals(other.getStartRow());
-      }
-      result = result && (hasStopRow() == other.hasStopRow());
-      if (hasStopRow()) {
-        result = result && getStopRow()
-            .equals(other.getStopRow());
-      }
-      result = result &&
-          getUnknownFields().equals(other.getUnknownFields());
-      return result;
-    }
-
-    private int memoizedHashCode = 0;
-    @java.lang.Override
-    public int hashCode() {
-      if (memoizedHashCode != 0) {
-        return memoizedHashCode;
-      }
-      int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
-      if (hasStartRow()) {
-        hash = (37 * hash) + STARTROW_FIELD_NUMBER;
-        hash = (53 * hash) + getStartRow().hashCode();
-      }
-      if (hasStopRow()) {
-        hash = (37 * hash) + STOPROW_FIELD_NUMBER;
-        hash = (53 * hash) + getStopRow().hashCode();
-      }
-      hash = (29 * hash) + getUnknownFields().hashCode();
-      memoizedHashCode = hash;
-      return hash;
-    }
-
-    public static org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest parseFrom(
-        byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest parseFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input);
-    }
-    public static org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest parseFrom(
-        java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input, extensionRegistry);
-    }
-    public static org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest parseDelimitedFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return PARSER.parseDelimitedFrom(input);
-    }
-    public static org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest parseDelimitedFrom(
-        java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseDelimitedFrom(input, extensionRegistry);
-    }
-    public static org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest parseFrom(
-        com.google.protobuf.CodedInputStream input)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input);
-    }
-    public static org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input, extensionRegistry);
-    }
-
-    public static Builder newBuilder() { return Builder.create(); }
-    public Builder newBuilderForType() { return newBuilder(); }
-    public static Builder newBuilder(org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest prototype) {
-      return newBuilder().mergeFrom(prototype);
-    }
-    public Builder toBuilder() { return newBuilder(this); }
-
-    @java.lang.Override
-    protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-      Builder builder = new Builder(parent);
-      return builder;
-    }
-    /**
-     * Protobuf type {@code StatCollectRequest}
-     */
-    public static final class Builder extends
-        com.google.protobuf.GeneratedMessage.Builder<Builder>
-       implements org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequestOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
-          getDescriptor() {
-        return org.apache.phoenix.coprocessor.generated.StatCollectorProtos.internal_static_StatCollectRequest_descriptor;
-      }
-
-      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
-          internalGetFieldAccessorTable() {
-        return org.apache.phoenix.coprocessor.generated.StatCollectorProtos.internal_static_StatCollectRequest_fieldAccessorTable
-            .ensureFieldAccessorsInitialized(
-                org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest.class, org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest.Builder.class);
-      }
-
-      // Construct using org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest.newBuilder()
-      private Builder() {
-        maybeForceBuilderInitialization();
-      }
-
-      private Builder(
-          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-        super(parent);
-        maybeForceBuilderInitialization();
-      }
-      private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
-        }
-      }
-      private static Builder create() {
-        return new Builder();
-      }
-
-      public Builder clear() {
-        super.clear();
-        startRow_ = com.google.protobuf.ByteString.EMPTY;
-        bitField0_ = (bitField0_ & ~0x00000001);
-        stopRow_ = com.google.protobuf.ByteString.EMPTY;
-        bitField0_ = (bitField0_ & ~0x00000002);
-        return this;
-      }
-
-      public Builder clone() {
-        return create().mergeFrom(buildPartial());
-      }
-
-      public com.google.protobuf.Descriptors.Descriptor
-          getDescriptorForType() {
-        return org.apache.phoenix.coprocessor.generated.StatCollectorProtos.internal_static_StatCollectRequest_descriptor;
-      }
-
-      public org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest getDefaultInstanceForType() {
-        return org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest.getDefaultInstance();
-      }
-
-      public org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest build() {
-        org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest result = buildPartial();
-        if (!result.isInitialized()) {
-          throw newUninitializedMessageException(result);
-        }
-        return result;
-      }
-
-      public org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest buildPartial() {
-        org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest result = new org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest(this);
-        int from_bitField0_ = bitField0_;
-        int to_bitField0_ = 0;
-        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
-          to_bitField0_ |= 0x00000001;
-        }
-        result.startRow_ = startRow_;
-        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
-          to_bitField0_ |= 0x00000002;
-        }
-        result.stopRow_ = stopRow_;
-        result.bitField0_ = to_bitField0_;
-        onBuilt();
-        return result;
-      }
-
-      public Builder mergeFrom(com.google.protobuf.Message other) {
-        if (other instanceof org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest) {
-          return mergeFrom((org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest)other);
-        } else {
-          super.mergeFrom(other);
-          return this;
-        }
-      }
-
-      public Builder mergeFrom(org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest other) {
-        if (other == org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest.getDefaultInstance()) return this;
-        if (other.hasStartRow()) {
-          setStartRow(other.getStartRow());
-        }
-        if (other.hasStopRow()) {
-          setStopRow(other.getStopRow());
-        }
-        this.mergeUnknownFields(other.getUnknownFields());
-        return this;
-      }
-
-      public final boolean isInitialized() {
-        return true;
-      }
-
-      public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws java.io.IOException {
-        org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest parsedMessage = null;
-        try {
-          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-          parsedMessage = (org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest) e.getUnfinishedMessage();
-          throw e;
-        } finally {
-          if (parsedMessage != null) {
-            mergeFrom(parsedMessage);
-          }
-        }
-        return this;
-      }
-      private int bitField0_;
-
-      // optional bytes startRow = 1;
-      private com.google.protobuf.ByteString startRow_ = com.google.protobuf.ByteString.EMPTY;
-      /**
-       * <code>optional bytes startRow = 1;</code>
-       */
-      public boolean hasStartRow() {
-        return ((bitField0_ & 0x00000001) == 0x00000001);
-      }
-      /**
-       * <code>optional bytes startRow = 1;</code>
-       */
-      public com.google.protobuf.ByteString getStartRow() {
-        return startRow_;
-      }
-      /**
-       * <code>optional bytes startRow = 1;</code>
-       */
-      public Builder setStartRow(com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000001;
-        startRow_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional bytes startRow = 1;</code>
-       */
-      public Builder clearStartRow() {
-        bitField0_ = (bitField0_ & ~0x00000001);
-        startRow_ = getDefaultInstance().getStartRow();
-        onChanged();
-        return this;
-      }
-
-      // optional bytes stopRow = 2;
-      private com.google.protobuf.ByteString stopRow_ = com.google.protobuf.ByteString.EMPTY;
-      /**
-       * <code>optional bytes stopRow = 2;</code>
-       */
-      public boolean hasStopRow() {
-        return ((bitField0_ & 0x00000002) == 0x00000002);
-      }
-      /**
-       * <code>optional bytes stopRow = 2;</code>
-       */
-      public com.google.protobuf.ByteString getStopRow() {
-        return stopRow_;
-      }
-      /**
-       * <code>optional bytes stopRow = 2;</code>
-       */
-      public Builder setStopRow(com.google.protobuf.ByteString value) {
-        if (value == null) {
-    throw new NullPointerException();
-  }
-  bitField0_ |= 0x00000002;
-        stopRow_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional bytes stopRow = 2;</code>
-       */
-      public Builder clearStopRow() {
-        bitField0_ = (bitField0_ & ~0x00000002);
-        stopRow_ = getDefaultInstance().getStopRow();
-        onChanged();
-        return this;
-      }
-
-      // @@protoc_insertion_point(builder_scope:StatCollectRequest)
-    }
-
-    static {
-      defaultInstance = new StatCollectRequest(true);
-      defaultInstance.initFields();
-    }
-
-    // @@protoc_insertion_point(class_scope:StatCollectRequest)
-  }
-
-  public interface StatCollectResponseOrBuilder
-      extends com.google.protobuf.MessageOrBuilder {
-
-    // required uint64 rowsScanned = 1;
-    /**
-     * <code>required uint64 rowsScanned = 1;</code>
-     */
-    boolean hasRowsScanned();
-    /**
-     * <code>required uint64 rowsScanned = 1;</code>
-     */
-    long getRowsScanned();
-  }
-  /**
-   * Protobuf type {@code StatCollectResponse}
-   */
-  public static final class StatCollectResponse extends
-      com.google.protobuf.GeneratedMessage
-      implements StatCollectResponseOrBuilder {
-    // Use StatCollectResponse.newBuilder() to construct.
-    private StatCollectResponse(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
-      super(builder);
-      this.unknownFields = builder.getUnknownFields();
-    }
-    private StatCollectResponse(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
-
-    private static final StatCollectResponse defaultInstance;
-    public static StatCollectResponse getDefaultInstance() {
-      return defaultInstance;
-    }
-
-    public StatCollectResponse getDefaultInstanceForType() {
-      return defaultInstance;
-    }
-
-    private final com.google.protobuf.UnknownFieldSet unknownFields;
-    @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
-        getUnknownFields() {
-      return this.unknownFields;
-    }
-    private StatCollectResponse(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      initFields();
-      int mutable_bitField0_ = 0;
-      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
-          com.google.protobuf.UnknownFieldSet.newBuilder();
-      try {
-        boolean done = false;
-        while (!done) {
-          int tag = input.readTag();
-          switch (tag) {
-            case 0:
-              done = true;
-              break;
-            default: {
-              if (!parseUnknownField(input, unknownFields,
-                                     extensionRegistry, tag)) {
-                done = true;
-              }
-              break;
-            }
-            case 8: {
-              bitField0_ |= 0x00000001;
-              rowsScanned_ = input.readUInt64();
-              break;
-            }
-          }
-        }
-      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-        throw e.setUnfinishedMessage(this);
-      } catch (java.io.IOException e) {
-        throw new com.google.protobuf.InvalidProtocolBufferException(
-            e.getMessage()).setUnfinishedMessage(this);
-      } finally {
-        this.unknownFields = unknownFields.build();
-        makeExtensionsImmutable();
-      }
-    }
-    public static final com.google.protobuf.Descriptors.Descriptor
-        getDescriptor() {
-      return org.apache.phoenix.coprocessor.generated.StatCollectorProtos.internal_static_StatCollectResponse_descriptor;
-    }
-
-    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
-        internalGetFieldAccessorTable() {
-      return org.apache.phoenix.coprocessor.generated.StatCollectorProtos.internal_static_StatCollectResponse_fieldAccessorTable
-          .ensureFieldAccessorsInitialized(
-              org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse.class, org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse.Builder.class);
-    }
-
-    public static com.google.protobuf.Parser<StatCollectResponse> PARSER =
-        new com.google.protobuf.AbstractParser<StatCollectResponse>() {
-      public StatCollectResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
-        return new StatCollectResponse(input, extensionRegistry);
-      }
-    };
-
-    @java.lang.Override
-    public com.google.protobuf.Parser<StatCollectResponse> getParserForType() {
-      return PARSER;
-    }
-
-    private int bitField0_;
-    // required uint64 rowsScanned = 1;
-    public static final int ROWSSCANNED_FIELD_NUMBER = 1;
-    private long rowsScanned_;
-    /**
-     * <code>required uint64 rowsScanned = 1;</code>
-     */
-    public boolean hasRowsScanned() {
-      return ((bitField0_ & 0x00000001) == 0x00000001);
-    }
-    /**
-     * <code>required uint64 rowsScanned = 1;</code>
-     */
-    public long getRowsScanned() {
-      return rowsScanned_;
-    }
-
-    private void initFields() {
-      rowsScanned_ = 0L;
-    }
-    private byte memoizedIsInitialized = -1;
-    public final boolean isInitialized() {
-      byte isInitialized = memoizedIsInitialized;
-      if (isInitialized != -1) return isInitialized == 1;
-
-      if (!hasRowsScanned()) {
-        memoizedIsInitialized = 0;
-        return false;
-      }
-      memoizedIsInitialized = 1;
-      return true;
-    }
-
-    public void writeTo(com.google.protobuf.CodedOutputStream output)
-                        throws java.io.IOException {
-      getSerializedSize();
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        output.writeUInt64(1, rowsScanned_);
-      }
-      getUnknownFields().writeTo(output);
-    }
-
-    private int memoizedSerializedSize = -1;
-    public int getSerializedSize() {
-      int size = memoizedSerializedSize;
-      if (size != -1) return size;
-
-      size = 0;
-      if (((bitField0_ & 0x00000001) == 0x00000001)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeUInt64Size(1, rowsScanned_);
-      }
-      size += getUnknownFields().getSerializedSize();
-      memoizedSerializedSize = size;
-      return size;
-    }
-
-    private static final long serialVersionUID = 0L;
-    @java.lang.Override
-    protected java.lang.Object writeReplace()
-        throws java.io.ObjectStreamException {
-      return super.writeReplace();
-    }
-
-    @java.lang.Override
-    public boolean equals(final java.lang.Object obj) {
-      if (obj == this) {
-       return true;
-      }
-      if (!(obj instanceof org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse)) {
-        return super.equals(obj);
-      }
-      org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse other = (org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse) obj;
-
-      boolean result = true;
-      result = result && (hasRowsScanned() == other.hasRowsScanned());
-      if (hasRowsScanned()) {
-        result = result && (getRowsScanned()
-            == other.getRowsScanned());
-      }
-      result = result &&
-          getUnknownFields().equals(other.getUnknownFields());
-      return result;
-    }
-
-    private int memoizedHashCode = 0;
-    @java.lang.Override
-    public int hashCode() {
-      if (memoizedHashCode != 0) {
-        return memoizedHashCode;
-      }
-      int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
-      if (hasRowsScanned()) {
-        hash = (37 * hash) + ROWSSCANNED_FIELD_NUMBER;
-        hash = (53 * hash) + hashLong(getRowsScanned());
-      }
-      hash = (29 * hash) + getUnknownFields().hashCode();
-      memoizedHashCode = hash;
-      return hash;
-    }
-
-    public static org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse parseFrom(
-        com.google.protobuf.ByteString data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse parseFrom(
-        byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse parseFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input);
-    }
-    public static org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse parseFrom(
-        java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input, extensionRegistry);
-    }
-    public static org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse parseDelimitedFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return PARSER.parseDelimitedFrom(input);
-    }
-    public static org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse parseDelimitedFrom(
-        java.io.InputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseDelimitedFrom(input, extensionRegistry);
-    }
-    public static org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input);
-    }
-    public static org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse parseFrom(
-        com.google.protobuf.CodedInputStream input,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input, extensionRegistry);
-    }
-
-    public static Builder newBuilder() { return Builder.create(); }
-    public Builder newBuilderForType() { return newBuilder(); }
-    public static Builder newBuilder(org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse prototype) {
-      return newBuilder().mergeFrom(prototype);
-    }
-    public Builder toBuilder() { return newBuilder(this); }
-
-    @java.lang.Override
-    protected Builder newBuilderForType(
-        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-      Builder builder = new Builder(parent);
-      return builder;
-    }
-    /**
-     * Protobuf type {@code StatCollectResponse}
-     */
-    public static final class Builder extends
-        com.google.protobuf.GeneratedMessage.Builder<Builder>
-       implements org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
-          getDescriptor() {
-        return org.apache.phoenix.coprocessor.generated.StatCollectorProtos.internal_static_StatCollectResponse_descriptor;
-      }
-
-      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
-          internalGetFieldAccessorTable() {
-        return org.apache.phoenix.coprocessor.generated.StatCollectorProtos.internal_static_StatCollectResponse_fieldAccessorTable
-            .ensureFieldAccessorsInitialized(
-                org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse.class, org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse.Builder.class);
-      }
-
-      // Construct using org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse.newBuilder()
-      private Builder() {
-        maybeForceBuilderInitialization();
-      }
-
-      private Builder(
-          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
-        super(parent);
-        maybeForceBuilderInitialization();
-      }
-      private void maybeForceBuilderInitialization() {
-        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
-        }
-      }
-      private static Builder create() {
-        return new Builder();
-      }
-
-      public Builder clear() {
-        super.clear();
-        rowsScanned_ = 0L;
-        bitField0_ = (bitField0_ & ~0x00000001);
-        return this;
-      }
-
-      public Builder clone() {
-        return create().mergeFrom(buildPartial());
-      }
-
-      public com.google.protobuf.Descriptors.Descriptor
-          getDescriptorForType() {
-        return org.apache.phoenix.coprocessor.generated.StatCollectorProtos.internal_static_StatCollectResponse_descriptor;
-      }
-
-      public org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse getDefaultInstanceForType() {
-        return org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse.getDefaultInstance();
-      }
-
-      public org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse build() {
-        org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse result = buildPartial();
-        if (!result.isInitialized()) {
-          throw newUninitializedMessageException(result);
-        }
-        return result;
-      }
-
-      public org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse buildPartial() {
-        org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse result = new org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse(this);
-        int from_bitField0_ = bitField0_;
-        int to_bitField0_ = 0;
-        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
-          to_bitField0_ |= 0x00000001;
-        }
-        result.rowsScanned_ = rowsScanned_;
-        result.bitField0_ = to_bitField0_;
-        onBuilt();
-        return result;
-      }
-
-      public Builder mergeFrom(com.google.protobuf.Message other) {
-        if (other instanceof org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse) {
-          return mergeFrom((org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse)other);
-        } else {
-          super.mergeFrom(other);
-          return this;
-        }
-      }
-
-      public Builder mergeFrom(org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse other) {
-        if (other == org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse.getDefaultInstance()) return this;
-        if (other.hasRowsScanned()) {
-          setRowsScanned(other.getRowsScanned());
-        }
-        this.mergeUnknownFields(other.getUnknownFields());
-        return this;
-      }
-
-      public final boolean isInitialized() {
-        if (!hasRowsScanned()) {
-          
-          return false;
-        }
-        return true;
-      }
-
-      public Builder mergeFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws java.io.IOException {
-        org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse parsedMessage = null;
-        try {
-          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-          parsedMessage = (org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse) e.getUnfinishedMessage();
-          throw e;
-        } finally {
-          if (parsedMessage != null) {
-            mergeFrom(parsedMessage);
-          }
-        }
-        return this;
-      }
-      private int bitField0_;
-
-      // required uint64 rowsScanned = 1;
-      private long rowsScanned_ ;
-      /**
-       * <code>required uint64 rowsScanned = 1;</code>
-       */
-      public boolean hasRowsScanned() {
-        return ((bitField0_ & 0x00000001) == 0x00000001);
-      }
-      /**
-       * <code>required uint64 rowsScanned = 1;</code>
-       */
-      public long getRowsScanned() {
-        return rowsScanned_;
-      }
-      /**
-       * <code>required uint64 rowsScanned = 1;</code>
-       */
-      public Builder setRowsScanned(long value) {
-        bitField0_ |= 0x00000001;
-        rowsScanned_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>required uint64 rowsScanned = 1;</code>
-       */
-      public Builder clearRowsScanned() {
-        bitField0_ = (bitField0_ & ~0x00000001);
-        rowsScanned_ = 0L;
-        onChanged();
-        return this;
-      }
-
-      // @@protoc_insertion_point(builder_scope:StatCollectResponse)
-    }
-
-    static {
-      defaultInstance = new StatCollectResponse(true);
-      defaultInstance.initFields();
-    }
-
-    // @@protoc_insertion_point(class_scope:StatCollectResponse)
-  }
-
-  /**
-   * Protobuf service {@code StatCollectService}
-   */
-  public static abstract class StatCollectService
-      implements com.google.protobuf.Service {
-    protected StatCollectService() {}
-
-    public interface Interface {
-      /**
-       * <code>rpc collectStat(.StatCollectRequest) returns (.StatCollectResponse);</code>
-       */
-      public abstract void collectStat(
-          com.google.protobuf.RpcController controller,
-          org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest request,
-          com.google.protobuf.RpcCallback<org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse> done);
-
-    }
-
-    public static com.google.protobuf.Service newReflectiveService(
-        final Interface impl) {
-      return new StatCollectService() {
-        @java.lang.Override
-        public  void collectStat(
-            com.google.protobuf.RpcController controller,
-            org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest request,
-            com.google.protobuf.RpcCallback<org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse> done) {
-          impl.collectStat(controller, request, done);
-        }
-
-      };
-    }
-
-    public static com.google.protobuf.BlockingService
-        newReflectiveBlockingService(final BlockingInterface impl) {
-      return new com.google.protobuf.BlockingService() {
-        public final com.google.protobuf.Descriptors.ServiceDescriptor
-            getDescriptorForType() {
-          return getDescriptor();
-        }
-
-        public final com.google.protobuf.Message callBlockingMethod(
-            com.google.protobuf.Descriptors.MethodDescriptor method,
-            com.google.protobuf.RpcController controller,
-            com.google.protobuf.Message request)
-            throws com.google.protobuf.ServiceException {
-          if (method.getService() != getDescriptor()) {
-            throw new java.lang.IllegalArgumentException(
-              "Service.callBlockingMethod() given method descriptor for " +
-              "wrong service type.");
-          }
-          switch(method.getIndex()) {
-            case 0:
-              return impl.collectStat(controller, (org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest)request);
-            default:
-              throw new java.lang.AssertionError("Can't get here.");
-          }
-        }
-
-        public final com.google.protobuf.Message
-            getRequestPrototype(
-            com.google.protobuf.Descriptors.MethodDescriptor method) {
-          if (method.getService() != getDescriptor()) {
-            throw new java.lang.IllegalArgumentException(
-              "Service.getRequestPrototype() given method " +
-              "descriptor for wrong service type.");
-          }
-          switch(method.getIndex()) {
-            case 0:
-              return org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest.getDefaultInstance();
-            default:
-              throw new java.lang.AssertionError("Can't get here.");
-          }
-        }
-
-        public final com.google.protobuf.Message
-            getResponsePrototype(
-            com.google.protobuf.Descriptors.MethodDescriptor method) {
-          if (method.getService() != getDescriptor()) {
-            throw new java.lang.IllegalArgumentException(
-              "Service.getResponsePrototype() given method " +
-              "descriptor for wrong service type.");
-          }
-          switch(method.getIndex()) {
-            case 0:
-              return org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse.getDefaultInstance();
-            default:
-              throw new java.lang.AssertionError("Can't get here.");
-          }
-        }
-
-      };
-    }
-
-    /**
-     * <code>rpc collectStat(.StatCollectRequest) returns (.StatCollectResponse);</code>
-     */
-    public abstract void collectStat(
-        com.google.protobuf.RpcController controller,
-        org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest request,
-        com.google.protobuf.RpcCallback<org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse> done);
-
-    public static final
-        com.google.protobuf.Descriptors.ServiceDescriptor
-        getDescriptor() {
-      return org.apache.phoenix.coprocessor.generated.StatCollectorProtos.getDescriptor().getServices().get(0);
-    }
-    public final com.google.protobuf.Descriptors.ServiceDescriptor
-        getDescriptorForType() {
-      return getDescriptor();
-    }
-
-    public final void callMethod(
-        com.google.protobuf.Descriptors.MethodDescriptor method,
-        com.google.protobuf.RpcController controller,
-        com.google.protobuf.Message request,
-        com.google.protobuf.RpcCallback<
-          com.google.protobuf.Message> done) {
-      if (method.getService() != getDescriptor()) {
-        throw new java.lang.IllegalArgumentException(
-          "Service.callMethod() given method descriptor for wrong " +
-          "service type.");
-      }
-      switch(method.getIndex()) {
-        case 0:
-          this.collectStat(controller, (org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest)request,
-            com.google.protobuf.RpcUtil.<org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse>specializeCallback(
-              done));
-          return;
-        default:
-          throw new java.lang.AssertionError("Can't get here.");
-      }
-    }
-
-    public final com.google.protobuf.Message
-        getRequestPrototype(
-        com.google.protobuf.Descriptors.MethodDescriptor method) {
-      if (method.getService() != getDescriptor()) {
-        throw new java.lang.IllegalArgumentException(
-          "Service.getRequestPrototype() given method " +
-          "descriptor for wrong service type.");
-      }
-      switch(method.getIndex()) {
-        case 0:
-          return org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest.getDefaultInstance();
-        default:
-          throw new java.lang.AssertionError("Can't get here.");
-      }
-    }
-
-    public final com.google.protobuf.Message
-        getResponsePrototype(
-        com.google.protobuf.Descriptors.MethodDescriptor method) {
-      if (method.getService() != getDescriptor()) {
-        throw new java.lang.IllegalArgumentException(
-          "Service.getResponsePrototype() given method " +
-          "descriptor for wrong service type.");
-      }
-      switch(method.getIndex()) {
-        case 0:
-          return org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse.getDefaultInstance();
-        default:
-          throw new java.lang.AssertionError("Can't get here.");
-      }
-    }
-
-    public static Stub newStub(
-        com.google.protobuf.RpcChannel channel) {
-      return new Stub(channel);
-    }
-
-    public static final class Stub extends org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectService implements Interface {
-      private Stub(com.google.protobuf.RpcChannel channel) {
-        this.channel = channel;
-      }
-
-      private final com.google.protobuf.RpcChannel channel;
-
-      public com.google.protobuf.RpcChannel getChannel() {
-        return channel;
-      }
-
-      public  void collectStat(
-          com.google.protobuf.RpcController controller,
-          org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest request,
-          com.google.protobuf.RpcCallback<org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse> done) {
-        channel.callMethod(
-          getDescriptor().getMethods().get(0),
-          controller,
-          request,
-          org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
-            done,
-            org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse.class,
-            org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse.getDefaultInstance()));
-      }
-    }
-
-    public static BlockingInterface newBlockingStub(
-        com.google.protobuf.BlockingRpcChannel channel) {
-      return new BlockingStub(channel);
-    }
-
-    public interface BlockingInterface {
-      public org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse collectStat(
-          com.google.protobuf.RpcController controller,
-          org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest request)
-          throws com.google.protobuf.ServiceException;
-    }
-
-    private static final class BlockingStub implements BlockingInterface {
-      private BlockingStub(com.google.protobuf.BlockingRpcChannel channel) {
-        this.channel = channel;
-      }
-
-      private final com.google.protobuf.BlockingRpcChannel channel;
-
-      public org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse collectStat(
-          com.google.protobuf.RpcController controller,
-          org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest request)
-          throws com.google.protobuf.ServiceException {
-        return (org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse) channel.callBlockingMethod(
-          getDescriptor().getMethods().get(0),
-          controller,
-          request,
-          org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse.getDefaultInstance());
-      }
-
-    }
-
-    // @@protoc_insertion_point(class_scope:StatCollectService)
-  }
-
-  private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_StatCollectRequest_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_StatCollectRequest_fieldAccessorTable;
-  private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_StatCollectResponse_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_StatCollectResponse_fieldAccessorTable;
-
-  public static com.google.protobuf.Descriptors.FileDescriptor
-      getDescriptor() {
-    return descriptor;
-  }
-  private static com.google.protobuf.Descriptors.FileDescriptor
-      descriptor;
-  static {
-    java.lang.String[] descriptorData = {
-      "\n\027StatisticsCollect.proto\"7\n\022StatCollect" +
-      "Request\022\020\n\010startRow\030\001 \001(\014\022\017\n\007stopRow\030\002 \001" +
-      "(\014\"*\n\023StatCollectResponse\022\023\n\013rowsScanned" +
-      "\030\001 \002(\0042N\n\022StatCollectService\0228\n\013collectS" +
-      "tat\022\023.StatCollectRequest\032\024.StatCollectRe" +
-      "sponseBG\n(org.apache.phoenix.coprocessor" +
-      ".generatedB\023StatCollectorProtosH\001\210\001\001\240\001\001"
-    };
-    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
-      new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
-        public com.google.protobuf.ExtensionRegistry assignDescriptors(
-            com.google.protobuf.Descriptors.FileDescriptor root) {
-          descriptor = root;
-          internal_static_StatCollectRequest_descriptor =
-            getDescriptor().getMessageTypes().get(0);
-          internal_static_StatCollectRequest_fieldAccessorTable = new
-            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
-              internal_static_StatCollectRequest_descriptor,
-              new java.lang.String[] { "StartRow", "StopRow", });
-          internal_static_StatCollectResponse_descriptor =
-            getDescriptor().getMessageTypes().get(1);
-          internal_static_StatCollectResponse_fieldAccessorTable = new
-            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
-              internal_static_StatCollectResponse_descriptor,
-              new java.lang.String[] { "RowsScanned", });
-          return null;
-        }
-      };
-    com.google.protobuf.Descriptors.FileDescriptor
-      .internalBuildGeneratedFileFrom(descriptorData,
-        new com.google.protobuf.Descriptors.FileDescriptor[] {
-        }, assigner);
-  }
-
-  // @@protoc_insertion_point(outer_class_scope)
-}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6a27cecc/phoenix-core/src/main/java/org/apache/phoenix/execute/ScanPlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/ScanPlan.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/ScanPlan.java
index 816cd1b..efb00a8 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/ScanPlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/ScanPlan.java
@@ -122,7 +122,7 @@ public class ScanPlan extends BaseQueryPlan {
         } else {
             // Region size estimated based on total number of bytes divided by number of regions
             long totByteSize = gpsInfo.getByteCount();
-            estRegionSize = totByteSize / (gpsInfo.getGuidePosts().size()+1);
+            estRegionSize = totByteSize / (gpsInfo.getGuidePostsCount()+1);
         }
         // TODO: configurable number of bytes?
         boolean isSerial = (perScanLimit * estRowSize < estRegionSize);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6a27cecc/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
----------------------------------------------------------------------
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 519b38f..b3235e2 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
@@ -22,6 +22,10 @@ import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_FAILED_QU
 import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_QUERY_TIMEOUT_COUNTER;
 import static org.apache.phoenix.util.ByteUtil.EMPTY_BYTE_ARRAY;
 
+import java.io.ByteArrayInputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.EOFException;
 import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -46,6 +50,7 @@ import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
 import org.apache.hadoop.hbase.filter.PageFilter;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.compile.QueryPlan;
@@ -75,7 +80,10 @@ import org.apache.phoenix.schema.StaleRegionBoundaryCacheException;
 import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.schema.stats.GuidePostsInfo;
 import org.apache.phoenix.schema.stats.PTableStats;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.PrefixByteCodec;
 import org.apache.phoenix.util.LogUtil;
+import org.apache.phoenix.util.PrefixByteDecoder;
 import org.apache.phoenix.util.SQLCloseables;
 import org.apache.phoenix.util.ScanUtil;
 import org.apache.phoenix.util.SchemaUtil;
@@ -356,22 +364,18 @@ public abstract class BaseResultIterators extends ExplainTable implements Result
         guideIndex = (guideIndex < 0 ? -(guideIndex + 1) : guideIndex);
         return guideIndex;
     }
-    
-    private List<byte[]> getGuidePosts(Set<byte[]> whereConditions) {
+
+    private GuidePostsInfo getGuidePosts(Set<byte[]> whereConditions) {
         /*
-         *  Don't use guide posts if:
-         *  1) We're doing a point lookup, as HBase is fast enough at those
-         *     to not need them to be further parallelized. TODO: pref test to verify
-         *  2) We're collecting stats, as in this case we need to scan entire
-         *     regions worth of data to track where to put the guide posts.
+         * Don't use guide posts if: 1) We're doing a point lookup, as HBase is fast enough at those to not need them to
+         * be further parallelized. TODO: pref test to verify 2) We're collecting stats, as in this case we need to scan
+         * entire regions worth of data to track where to put the guide posts.
          */
-        if (!useStats()) {
-            return Collections.emptyList();
-        }
-        
-        List<byte[]> gps = null;
+        if (!useStats()) { return GuidePostsInfo.EMPTY_GUIDEPOST; }
+
+        GuidePostsInfo gps = null;
         PTable table = getTable();
-        Map<byte[],GuidePostsInfo> guidePostMap = tableStats.getGuidePosts();
+        Map<byte[], GuidePostsInfo> guidePostMap = tableStats.getGuidePosts();
         byte[] defaultCF = SchemaUtil.getEmptyColumnFamily(getTable());
         if (table.getColumnFamilies().isEmpty()) {
             // For sure we can get the defaultCF from the table
@@ -379,37 +383,35 @@ public abstract class BaseResultIterators extends ExplainTable implements Result
         } else {
             byte[] familyInWhere = null;
             if (!whereConditions.isEmpty()) {
-              if (whereConditions.contains(defaultCF)) {
-                gps = getDefaultFamilyGuidePosts(guidePostMap, defaultCF);
-              } else {
-                familyInWhere = whereConditions.iterator().next();
-                if(familyInWhere != null) {
-                  GuidePostsInfo guidePostsInfo = guidePostMap.get(familyInWhere);
-                  if (guidePostsInfo != null) {
-                      gps = guidePostsInfo.getGuidePosts();
-                  } else {
-                    // As there are no guideposts collected for the where family we go with the default CF
+                if (whereConditions.contains(defaultCF)) {
                     gps = getDefaultFamilyGuidePosts(guidePostMap, defaultCF);
-                  }
+                } else {
+                    familyInWhere = whereConditions.iterator().next();
+                    if (familyInWhere != null) {
+                        GuidePostsInfo guidePostsInfo = guidePostMap.get(familyInWhere);
+                        if (guidePostsInfo != null) {
+                            gps = guidePostsInfo;
+                        } else {
+                            // As there are no guideposts collected for the where family we go with the default CF
+                            gps = getDefaultFamilyGuidePosts(guidePostMap, defaultCF);
+                        }
+                    }
                 }
-              }
             } else {
-              gps = getDefaultFamilyGuidePosts(guidePostMap, defaultCF);
+                gps = getDefaultFamilyGuidePosts(guidePostMap, defaultCF);
             }
         }
-        if (gps == null) {
-            return Collections.emptyList();
-        }
+        if (gps == null) { return GuidePostsInfo.EMPTY_GUIDEPOST; }
         return gps;
     }
 
-    private List<byte[]> getDefaultFamilyGuidePosts(Map<byte[], GuidePostsInfo> guidePostMap, byte[] defaultCF) {
-      if (guidePostMap.get(defaultCF) != null) {
-          return guidePostMap.get(defaultCF).getGuidePosts();
-      }
-      return null;
+    private GuidePostsInfo getDefaultFamilyGuidePosts(Map<byte[], GuidePostsInfo> guidePostMap, byte[] defaultCF) {
+        if (guidePostMap.get(defaultCF) != null) {
+                return guidePostMap.get(defaultCF);
+        }
+        return null;
     }
-    
+
     private static String toString(List<byte[]> gps) {
         StringBuilder buf = new StringBuilder(gps.size() * 100);
         buf.append("[");
@@ -461,10 +463,7 @@ public abstract class BaseResultIterators extends ExplainTable implements Result
         for(Pair<byte[], byte[]> where : context.getWhereConditionColumns()) {
           whereConditions.add(where.getFirst());
         }
-        List<byte[]> gps = getGuidePosts(whereConditions);
-        if (logger.isDebugEnabled()) {
-            logger.debug("Guideposts: " + toString(gps));
-        }
+        GuidePostsInfo gps = getGuidePosts(whereConditions);
         boolean traverseAllRegions = isSalted || isLocalIndex;
         if (!traverseAllRegions) {
             byte[] scanStartRow = scan.getStartRow();
@@ -490,15 +489,35 @@ public abstract class BaseResultIterators extends ExplainTable implements Result
         }
         List<List<Scan>> parallelScans = Lists.newArrayListWithExpectedSize(stopIndex - regionIndex + 1);
         
-        byte[] currentKey = startKey;
-        int guideIndex = currentKey.length == 0 ? 0 : getIndexContainingInclusive(gps, currentKey);
-        int gpsSize = gps.size();
+        ImmutableBytesWritable currentKey = new ImmutableBytesWritable(startKey);
+        
+        int gpsSize = gps.getGuidePostsCount();
         int estGuidepostsPerRegion = gpsSize == 0 ? 1 : gpsSize / regionLocations.size() + 1;
         int keyOffset = 0;
+        ImmutableBytesWritable currentGuidePost = ByteUtil.EMPTY_IMMUTABLE_BYTE_ARRAY;
         List<Scan> scans = Lists.newArrayListWithExpectedSize(estGuidepostsPerRegion);
+        ImmutableBytesWritable guidePosts = gps.getGuidePosts();
+        ByteArrayInputStream stream = null;
+        DataInput input = null;
+        PrefixByteDecoder decoder = null;
+        int guideIndex = 0;
+        if (gpsSize > 0) {
+            stream = new ByteArrayInputStream(guidePosts.get(), guidePosts.getOffset(), guidePosts.getLength());
+            input = new DataInputStream(stream);
+            decoder = new PrefixByteDecoder(gps.getMaxLength());
+            try {
+                while (currentKey.compareTo(currentGuidePost = PrefixByteCodec.decode(decoder, input)) >= 0
+                        && currentKey.getLength() != 0) {
+                    guideIndex++;
+                }
+            } catch (EOFException e) {}
+        }
+        byte[] currentKeyBytes = currentKey.copyBytes();
+
         // Merge bisect with guideposts for all but the last region
         while (regionIndex <= stopIndex) {
-            byte[] currentGuidePost, endKey, endRegionKey = EMPTY_BYTE_ARRAY;
+            byte[] currentGuidePostBytes = currentGuidePost.copyBytes();
+            byte[] endKey, endRegionKey = EMPTY_BYTE_ARRAY;
             if (regionIndex == stopIndex) {
                 endKey = stopKey;
             } else {
@@ -510,14 +529,18 @@ public abstract class BaseResultIterators extends ExplainTable implements Result
                 endRegionKey = regionInfo.getEndKey();
                 keyOffset = ScanUtil.getRowKeyOffset(regionInfo.getStartKey(), endRegionKey);
             }
-            while (guideIndex < gpsSize
-                    && (Bytes.compareTo(currentGuidePost = gps.get(guideIndex), endKey) <= 0 || endKey.length == 0)) {
-                Scan newScan = scanRanges.intersectScan(scan, currentKey, currentGuidePost, keyOffset, false);
-                scans = addNewScan(parallelScans, scans, newScan, currentGuidePost, false, regionLocation);
-                currentKey = currentGuidePost;
-                guideIndex++;
-            }
-            Scan newScan = scanRanges.intersectScan(scan, currentKey, endKey, keyOffset, true);
+            try {
+                while (guideIndex < gpsSize && (currentGuidePost.compareTo(endKey) <= 0 || endKey.length == 0)) {
+                    Scan newScan = scanRanges.intersectScan(scan, currentKeyBytes, currentGuidePostBytes, keyOffset,
+                            false);
+                    scans = addNewScan(parallelScans, scans, newScan, currentGuidePostBytes, false, regionLocation);
+                    currentKeyBytes = currentGuidePost.copyBytes();
+                    currentGuidePost = PrefixByteCodec.decode(decoder, input);
+                    currentGuidePostBytes = currentGuidePost.copyBytes();
+                    guideIndex++;
+                }
+            } catch (EOFException e) {}
+            Scan newScan = scanRanges.intersectScan(scan, currentKeyBytes, endKey, keyOffset, true);
             if (isLocalIndex) {
                 if (newScan != null) {
                     newScan.setAttribute(EXPECTED_UPPER_REGION_KEY, endRegionKey);
@@ -526,15 +549,17 @@ public abstract class BaseResultIterators extends ExplainTable implements Result
                 }
             }
             scans = addNewScan(parallelScans, scans, newScan, endKey, true, regionLocation);
-            currentKey = endKey;
+            currentKeyBytes = endKey;
             regionIndex++;
         }
         if (!scans.isEmpty()) { // Add any remaining scans
             parallelScans.add(scans);
         }
+        PrefixByteCodec.close(stream);
         return parallelScans;
     }
 
+   
     public static <T> List<T> reverseIfNecessary(List<T> list, boolean reverse) {
         if (!reverse) {
             return list;
@@ -805,8 +830,9 @@ public abstract class BaseResultIterators extends ExplainTable implements Result
         explain(buf.toString(),planSteps);
     }
 
-	@Override
-	public String toString() {
-		return "ResultIterators [name=" + getName() + ",id=" + scanId + ",scans=" + scans + "]";
-	}
+    @Override
+    public String toString() {
+        return "ResultIterators [name=" + getName() + ",id=" + scanId + ",scans=" + scans + "]";
+    }
+
 }
\ No newline at end of file


[09/50] [abbrv] phoenix git commit: PHOENIX-2611 current_date/time functions produce invalid values

Posted by ma...@apache.org.
PHOENIX-2611 current_date/time functions produce invalid values


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

Branch: refs/heads/calcite
Commit: 15432d1ac63ef429600333ad36c06f8e43deacf8
Parents: f572fa6
Author: James Taylor <jt...@salesforce.com>
Authored: Tue Jan 19 20:25:55 2016 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Tue Jan 19 20:44:51 2016 -0800

----------------------------------------------------------------------
 .../it/java/org/apache/phoenix/end2end/DateTimeIT.java    | 10 ++++++++++
 .../src/main/java/org/apache/phoenix/schema/TableRef.java |  3 +--
 2 files changed, 11 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/15432d1a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DateTimeIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DateTimeIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DateTimeIT.java
index 0fce403..6c53f1d 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DateTimeIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DateTimeIT.java
@@ -52,6 +52,7 @@ import java.util.Calendar;
 import java.util.GregorianCalendar;
 
 import org.apache.phoenix.util.DateUtil;
+import org.apache.phoenix.util.TestUtil;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -664,4 +665,13 @@ public class DateTimeIT extends BaseHBaseManagedTimeIT {
         assertNull(rs.getDate(1, GregorianCalendar.getInstance()));
         assertFalse(rs.next());
     }
+    
+    @Test
+    public void testCurrentDateWithNoTable() throws Exception {
+        long expectedTime = System.currentTimeMillis();
+        ResultSet rs = conn.createStatement().executeQuery("SELECT CURRENT_DATE()");
+        assertTrue(rs.next());
+        long actualTime = rs.getDate(1).getTime();
+        assertTrue(Math.abs(actualTime - expectedTime) < TestUtil.MILLIS_IN_DAY);
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/15432d1a/phoenix-core/src/main/java/org/apache/phoenix/schema/TableRef.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/TableRef.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/TableRef.java
index 8f6e271..e231342 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/TableRef.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/TableRef.java
@@ -19,7 +19,6 @@ package org.apache.phoenix.schema;
 
 import java.util.Objects;
 
-import org.apache.hadoop.hbase.HConstants;
 import org.apache.phoenix.compile.TupleProjectionCompiler;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.util.IndexUtil;
@@ -49,7 +48,7 @@ public class TableRef {
     }
     
     public TableRef(PTable table) {
-        this(null, table, HConstants.LATEST_TIMESTAMP, false);
+        this(null, table, QueryConstants.UNSET_TIMESTAMP, false);
     }
     
     public TableRef(PTable table, long upperBoundTimeStamp, long lowerBoundTimeStamp) {


[18/50] [abbrv] phoenix git commit: PHOENIX-2616 Indexes over immutable tables not marked as immutable

Posted by ma...@apache.org.
PHOENIX-2616 Indexes over immutable tables not marked as immutable


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

Branch: refs/heads/calcite
Commit: c966935921a85a0e6b5f4ba49d26e167f08349ed
Parents: 6911770
Author: James Taylor <ja...@apache.org>
Authored: Wed Jan 20 18:33:12 2016 -0800
Committer: James Taylor <ja...@apache.org>
Committed: Wed Jan 20 18:33:47 2016 -0800

----------------------------------------------------------------------
 .../apache/phoenix/execute/MutationState.java   | 18 ++++----
 .../query/ConnectionQueryServicesImpl.java      | 43 ++++++++++++++++----
 .../apache/phoenix/schema/MetaDataClient.java   |  5 ++-
 .../phoenix/compile/QueryCompilerTest.java      | 12 +++---
 4 files changed, 54 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/c9669359/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
index f1a9c02..0a5b053 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
@@ -92,12 +92,6 @@ import org.apache.phoenix.util.TransactionUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.base.Predicate;
-import com.google.common.collect.Iterators;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-
 import co.cask.tephra.Transaction;
 import co.cask.tephra.Transaction.VisibilityLevel;
 import co.cask.tephra.TransactionAware;
@@ -110,12 +104,16 @@ import co.cask.tephra.hbase11.TransactionAwareHTable;
 import co.cask.tephra.visibility.FenceWait;
 import co.cask.tephra.visibility.VisibilityFence;
 
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
 /**
  * 
  * Tracks the uncommitted state
  *
- * 
- * @since 0.1
  */
 public class MutationState implements SQLCloseable {
     private static final Logger logger = LoggerFactory.getLogger(MutationState.class);
@@ -1227,7 +1225,9 @@ public class MutationState implements SQLCloseable {
             List<PTable> oldIndexes;
             PTableRef ptableRef = cache.getTableRef(dataTable.getKey());
             oldIndexes = ptableRef.getTable().getIndexes();
-            MetaDataMutationResult result = client.updateCache(dataTable.getTenantId(), dataTable.getSchemaName().getString(), dataTable.getTableName().getString());
+            // Always check at server for metadata change, as it's possible that the table is configured to not check for metadata changes
+            // but in this case, the tx manager is telling us it's likely that there has been a change.
+            MetaDataMutationResult result = client.updateCache(dataTable.getTenantId(), dataTable.getSchemaName().getString(), dataTable.getTableName().getString(), true);
             long timestamp = TransactionUtil.getResolvedTime(connection, result);
             tableRef.setTimeStamp(timestamp);
             PTable updatedDataTable = result.getTable();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c9669359/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 4522cf8..73999b8 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
@@ -187,6 +187,12 @@ import org.apache.twill.zookeeper.ZKClients;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import co.cask.tephra.TransactionSystemClient;
+import co.cask.tephra.TxConstants;
+import co.cask.tephra.distributed.PooledClientProvider;
+import co.cask.tephra.distributed.TransactionServiceClient;
+import co.cask.tephra.hbase11.coprocessor.TransactionProcessor;
+
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Joiner;
 import com.google.common.base.Throwables;
@@ -199,12 +205,6 @@ import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 
-import co.cask.tephra.TransactionSystemClient;
-import co.cask.tephra.TxConstants;
-import co.cask.tephra.distributed.PooledClientProvider;
-import co.cask.tephra.distributed.TransactionServiceClient;
-import co.cask.tephra.hbase11.coprocessor.TransactionProcessor;
-
 
 public class ConnectionQueryServicesImpl extends DelegateQueryServices implements ConnectionQueryServices {
     private static final Logger logger = LoggerFactory.getLogger(ConnectionQueryServicesImpl.class);
@@ -2423,6 +2423,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                             PhoenixDatabaseMetaData.TRANSACTIONAL + " " + PBoolean.INSTANCE.getSqlTypeName());
                                     metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG, MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 1, 
                                             PhoenixDatabaseMetaData.UPDATE_CACHE_FREQUENCY + " " + PLong.INSTANCE.getSqlTypeName());
+                                    setImmutableTableIndexesImmutable(metaConnection);
 									// Drop old stats table so that new stats table is created
 									metaConnection = dropStatsTable(metaConnection,
 											MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0);
@@ -2527,7 +2528,35 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
     }
 
-	private PhoenixConnection dropStatsTable(PhoenixConnection oldMetaConnection, long timestamp)
+
+    /**
+     * Set IMMUTABLE_ROWS to true for all index tables over immutable tables.
+     * @param metaConnection connection over which to run the upgrade
+     * @throws SQLException
+     */
+    private static void setImmutableTableIndexesImmutable(PhoenixConnection metaConnection) throws SQLException {
+        boolean autoCommit = metaConnection.getAutoCommit();
+        try {
+            metaConnection.setAutoCommit(true);
+            metaConnection.createStatement().execute(
+                    "UPSERT INTO SYSTEM.CATALOG(TENANT_ID, TABLE_SCHEM, TABLE_NAME, COLUMN_NAME, COLUMN_FAMILY, IMMUTABLE_ROWS)\n" + 
+                    "SELECT A.TENANT_ID, A.TABLE_SCHEM,B.COLUMN_FAMILY,null,null,true\n" + 
+                    "FROM SYSTEM.CATALOG A JOIN SYSTEM.CATALOG B ON (\n" + 
+                    " A.TENANT_ID = B.TENANT_ID AND \n" + 
+                    " A.TABLE_SCHEM = B.TABLE_SCHEM AND\n" + 
+                    " A.TABLE_NAME = B.TABLE_NAME AND\n" + 
+                    " A.COLUMN_NAME = B.COLUMN_NAME AND\n" + 
+                    " B.LINK_TYPE = 1\n" + 
+                    ")\n" + 
+                    "WHERE A.COLUMN_FAMILY IS NULL AND\n" + 
+                    " B.COLUMN_FAMILY IS NOT NULL AND\n" + 
+                    " A.IMMUTABLE_ROWS = TRUE;");
+        } finally {
+            metaConnection.setAutoCommit(autoCommit);
+        }
+    }
+
+    private PhoenixConnection dropStatsTable(PhoenixConnection oldMetaConnection, long timestamp)
 			throws SQLException, IOException {
 		Properties props = PropertiesUtil.deepCopy(oldMetaConnection.getClientInfo());
 		props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(timestamp-1));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c9669359/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 064007f..d559842 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
@@ -114,6 +114,8 @@ import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
 
+import co.cask.tephra.TxConstants;
+
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
@@ -216,8 +218,6 @@ import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 import com.google.common.primitives.Ints;
 
-import co.cask.tephra.TxConstants;
-
 public class MetaDataClient {
     private static final Logger logger = LoggerFactory.getLogger(MetaDataClient.class);
 
@@ -1580,6 +1580,7 @@ public class MetaDataClient {
             if (parent != null && tableType == PTableType.INDEX) {
                 timestamp = TransactionUtil.getTableTimestamp(connection, transactional);
                 storeNulls = parent.getStoreNulls();
+                isImmutableRows = parent.isImmutableRows();
                 // Index on view
                 // TODO: Can we support a multi-tenant index directly on a multi-tenant
                 // table instead of only a view? We don't have anywhere to put the link

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c9669359/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
index 6d6dcdf..b3baa73 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
@@ -1189,7 +1189,7 @@ public class QueryCompilerTest extends BaseConnectionlessQueryTest {
             conn.createStatement().execute(ddl);
             assertImmutableRows(conn, "T", true);
             conn.createStatement().execute(indexDDL);
-            assertImmutableRows(conn, "T", true);
+            assertImmutableRows(conn, "I", true);
             conn.createStatement().execute("DELETE FROM t WHERE v2 = 'foo'");
             fail();
         } catch (SQLException e) {
@@ -1209,13 +1209,13 @@ public class QueryCompilerTest extends BaseConnectionlessQueryTest {
     
     @Test
     public void testInvalidNegativeArrayIndex() throws Exception {
-    	String query = "SELECT a_double_array[-20] FROM table_with_array";
-    	Connection conn = DriverManager.getConnection(getUrl());
+        String query = "SELECT a_double_array[-20] FROM table_with_array";
+        Connection conn = DriverManager.getConnection(getUrl());
         try {
             conn.createStatement().execute(query);
             fail();
         } catch (Exception e) {
-        	
+            
         }
     }
     @Test
@@ -1232,8 +1232,8 @@ public class QueryCompilerTest extends BaseConnectionlessQueryTest {
     
     @Test
     public void testNonArrayColumnWithIndex() throws Exception {
-    	String query = "SELECT a_float[1] FROM table_with_array";
-    	Connection conn = DriverManager.getConnection(getUrl());
+        String query = "SELECT a_float[1] FROM table_with_array";
+        Connection conn = DriverManager.getConnection(getUrl());
         try {
             conn.createStatement().execute(query);
             fail();


[21/50] [abbrv] phoenix git commit: PHOENIX-2612 RuleGeneartorTest updated with right pattern (Karan Singhal)

Posted by ma...@apache.org.
PHOENIX-2612 RuleGeneartorTest updated with right pattern (Karan Singhal)


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

Branch: refs/heads/calcite
Commit: dab79cbe0e4d10e1c259198de8e95b7e93d66b57
Parents: d345b88
Author: James Taylor <jt...@salesforce.com>
Authored: Wed Jan 20 21:06:32 2016 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Wed Jan 20 21:06:32 2016 -0800

----------------------------------------------------------------------
 .../main/java/org/apache/phoenix/pherf/PherfConstants.java    | 1 +
 .../src/test/java/org/apache/phoenix/pherf/ResourceTest.java  | 2 +-
 .../test/java/org/apache/phoenix/pherf/RuleGeneratorTest.java | 7 +++----
 3 files changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/dab79cbe/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/PherfConstants.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/PherfConstants.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/PherfConstants.java
index 42b5d21..bbae6ea 100644
--- a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/PherfConstants.java
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/PherfConstants.java
@@ -47,6 +47,7 @@ public class PherfConstants {
             SCENARIO_ROOT_PATTERN =
             		".*" + PherfConstants.RESOURCE_SCENARIO.substring(1) + ".*" + PherfConstants.RESOURCE_SCENARIO.substring(1) + ".*";
     public static final String SCHEMA_ROOT_PATTERN = ".*";
+    public static final String TEST_SCENARIO_ROOT_PATTERN = ".*" + "test" + "_" + PherfConstants.RESOURCE_SCENARIO.substring(1);
     public static final String PHERF_PROPERTIES = "pherf.properties";
 
     public static final String EXPORT_DIR = "CSV_EXPORT";

http://git-wip-us.apache.org/repos/asf/phoenix/blob/dab79cbe/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ResourceTest.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ResourceTest.java b/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ResourceTest.java
index cd567cb..d8cad48 100644
--- a/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ResourceTest.java
+++ b/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ResourceTest.java
@@ -43,7 +43,7 @@ public class ResourceTest {
     @Test
     public void testScenarioResourceList() throws Exception {
         String extension = ".xml";
-        assertResources(PherfConstants.SCENARIO_ROOT_PATTERN + extension, PherfConstants.RESOURCE_SCENARIO, extension);
+        assertResources(PherfConstants.TEST_SCENARIO_ROOT_PATTERN + extension, PherfConstants.RESOURCE_SCENARIO, extension);
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/dab79cbe/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/RuleGeneratorTest.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/RuleGeneratorTest.java b/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/RuleGeneratorTest.java
index 71b785b..b26b2f7 100644
--- a/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/RuleGeneratorTest.java
+++ b/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/RuleGeneratorTest.java
@@ -47,9 +47,8 @@ import org.junit.Ignore;
 import org.junit.Test;
 
 public class RuleGeneratorTest {
-    private static final String matcherScenario = PherfConstants.SCENARIO_ROOT_PATTERN + ".xml";
+    private static final String matcherScenario = PherfConstants.TEST_SCENARIO_ROOT_PATTERN + ".xml";
 
-    @Ignore("Flapping - see PHOENIX-2592")
     @Test
     public void testDateGenerator() throws Exception {
         XMLConfigParser parser = new XMLConfigParser(matcherScenario);
@@ -252,7 +251,7 @@ public class RuleGeneratorTest {
         expectedValues.add("bBByYhnNbBs9kWu");
         expectedValues.add("cCCyYhnNbBs9kWr");
 
-        XMLConfigParser parser = new XMLConfigParser(".*test_scenario.xml");
+        XMLConfigParser parser = new XMLConfigParser(matcherScenario);
         WriteWorkload loader = new WriteWorkload(parser);
         RulesApplier rulesApplier = loader.getRulesApplier();
         Scenario scenario = parser.getScenarios().get(0);
@@ -270,7 +269,7 @@ public class RuleGeneratorTest {
 
     @Test
     public void testRuleOverrides() throws Exception {
-        XMLConfigParser parser = new XMLConfigParser(".*test_scenario.xml");
+        XMLConfigParser parser = new XMLConfigParser(matcherScenario);
         WriteWorkload loader = new WriteWorkload(parser);
         RulesApplier rulesApplier = loader.getRulesApplier();
         Scenario scenario = parser.getScenarios().get(0);


[22/50] [abbrv] phoenix git commit: PHOENIX-2616 Indexes over immutable tables not marked as immutable

Posted by ma...@apache.org.
PHOENIX-2616 Indexes over immutable tables not marked as immutable


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

Branch: refs/heads/calcite
Commit: 6251f8f8c035ecf50051b974ca99499913e001c1
Parents: dab79cb
Author: James Taylor <jt...@salesforce.com>
Authored: Wed Jan 20 21:40:00 2016 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Wed Jan 20 21:40:00 2016 -0800

----------------------------------------------------------------------
 .../phoenix/query/ConnectionQueryServicesImpl.java  | 16 +++++++++-------
 1 file changed, 9 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/6251f8f8/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 e8c5e1a..cf21452 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
@@ -187,12 +187,6 @@ import org.apache.twill.zookeeper.ZKClients;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import co.cask.tephra.TransactionSystemClient;
-import co.cask.tephra.TxConstants;
-import co.cask.tephra.distributed.PooledClientProvider;
-import co.cask.tephra.distributed.TransactionServiceClient;
-import co.cask.tephra.hbase11.coprocessor.TransactionProcessor;
-
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Joiner;
 import com.google.common.base.Throwables;
@@ -205,6 +199,12 @@ import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 
+import co.cask.tephra.TransactionSystemClient;
+import co.cask.tephra.TxConstants;
+import co.cask.tephra.distributed.PooledClientProvider;
+import co.cask.tephra.distributed.TransactionServiceClient;
+import co.cask.tephra.hbase11.coprocessor.TransactionProcessor;
+
 
 public class ConnectionQueryServicesImpl extends DelegateQueryServices implements ConnectionQueryServices {
     private static final Logger logger = LoggerFactory.getLogger(ConnectionQueryServicesImpl.class);
@@ -2259,6 +2259,8 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         SQLException sqlE = null;
         try {
             metaConnection.createStatement().executeUpdate("ALTER TABLE " + tableName + " ADD " + (addIfNotExists ? " IF NOT EXISTS " : "") + columns );
+        } catch (NewerTableAlreadyExistsException e) {
+            logger.warn("Table already modified at this timestamp, so assuming add of these columns already done: " + columns);
         } catch (SQLException e) {
             logger.warn("Add column failed due to:" + e);
             sqlE = e;
@@ -2553,7 +2555,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     ")\n" + 
                     "WHERE A.COLUMN_FAMILY IS NULL AND\n" + 
                     " B.COLUMN_FAMILY IS NOT NULL AND\n" + 
-                    " A.IMMUTABLE_ROWS = TRUE;");
+                    " A.IMMUTABLE_ROWS = TRUE");
         } finally {
             metaConnection.setAutoCommit(autoCommit);
         }


[39/50] [abbrv] phoenix git commit: PHOENIX-2542 - CSV bulk loading with --schema option is broken

Posted by ma...@apache.org.
PHOENIX-2542 - CSV bulk loading with --schema option is broken


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

Branch: refs/heads/calcite
Commit: a6646d7ebdac53e4b011c6c6b57b8fc5b679d3ff
Parents: aaab87a
Author: ravimagham <ra...@bazaarvoice.com>
Authored: Thu Jan 28 07:42:04 2016 -0800
Committer: ravimagham <ra...@bazaarvoice.com>
Committed: Thu Jan 28 07:42:04 2016 -0800

----------------------------------------------------------------------
 .../it/java/org/apache/phoenix/end2end/CsvBulkLoadToolIT.java   | 5 +++--
 .../java/org/apache/phoenix/mapreduce/AbstractBulkLoadTool.java | 2 +-
 2 files changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6646d7e/phoenix-core/src/it/java/org/apache/phoenix/end2end/CsvBulkLoadToolIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CsvBulkLoadToolIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CsvBulkLoadToolIT.java
index 87652ab..6bc03bf 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CsvBulkLoadToolIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CsvBulkLoadToolIT.java
@@ -95,7 +95,7 @@ public class CsvBulkLoadToolIT {
     public void testBasicImport() throws Exception {
 
         Statement stmt = conn.createStatement();
-        stmt.execute("CREATE TABLE TABLE1 (ID INTEGER NOT NULL PRIMARY KEY, NAME VARCHAR, T DATE) SPLIT ON (1,2)");
+        stmt.execute("CREATE TABLE S.TABLE1 (ID INTEGER NOT NULL PRIMARY KEY, NAME VARCHAR, T DATE) SPLIT ON (1,2)");
 
         FileSystem fs = FileSystem.get(hbaseTestUtil.getConfiguration());
         FSDataOutputStream outputStream = fs.create(new Path("/tmp/input1.csv"));
@@ -110,10 +110,11 @@ public class CsvBulkLoadToolIT {
         int exitCode = csvBulkLoadTool.run(new String[] {
                 "--input", "/tmp/input1.csv",
                 "--table", "table1",
+                "--schema", "s",
                 "--zookeeper", zkQuorum});
         assertEquals(0, exitCode);
 
-        ResultSet rs = stmt.executeQuery("SELECT id, name, t FROM table1 ORDER BY id");
+        ResultSet rs = stmt.executeQuery("SELECT id, name, t FROM s.table1 ORDER BY id");
         assertTrue(rs.next());
         assertEquals(1, rs.getInt(1));
         assertEquals("Name 1", rs.getString(2));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6646d7e/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/AbstractBulkLoadTool.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/AbstractBulkLoadTool.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/AbstractBulkLoadTool.java
index 4b5d618..f6ba5f6 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/AbstractBulkLoadTool.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/AbstractBulkLoadTool.java
@@ -208,7 +208,7 @@ public abstract class AbstractBulkLoadTool extends Configured implements Tool {
         FormatToKeyValueMapper.configureColumnInfoList(conf, importColumns);
         boolean ignoreInvalidRows = cmdLine.hasOption(IGNORE_ERRORS_OPT.getOpt());
         conf.setBoolean(FormatToKeyValueMapper.IGNORE_INVALID_ROW_CONFKEY, ignoreInvalidRows);
-        conf.set(FormatToKeyValueMapper.TABLE_NAME_CONFKEY, tableName);
+        conf.set(FormatToKeyValueMapper.TABLE_NAME_CONFKEY, qualifiedTableName);
 
         // give subclasses their hook
         configureOptions(cmdLine, importColumns, conf);


[32/50] [abbrv] phoenix git commit: PHOENIX-2622 Writes to transactional table get slower after regions splits to multiple region servers

Posted by ma...@apache.org.
PHOENIX-2622 Writes to transactional table get slower after regions splits to multiple region servers


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

Branch: refs/heads/calcite
Commit: 424083d9f95e2e26c40b2d443c29392dda3cc084
Parents: 81b644f
Author: James Taylor <ja...@apache.org>
Authored: Fri Jan 22 17:21:58 2016 -0800
Committer: James Taylor <ja...@apache.org>
Committed: Fri Jan 22 17:25:03 2016 -0800

----------------------------------------------------------------------
 .../apache/phoenix/execute/MutationState.java   | 29 +++++++++++++-------
 .../apache/phoenix/jdbc/PhoenixStatement.java   |  1 -
 2 files changed, 19 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/424083d9/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
index 46aa819..1658962 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
@@ -54,7 +54,6 @@ import org.apache.phoenix.coprocessor.MetaDataProtocol.MetaDataMutationResult;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
-import org.apache.phoenix.hbase.index.util.KeyValueBuilder;
 import org.apache.phoenix.index.IndexMaintainer;
 import org.apache.phoenix.index.IndexMetaDataCacheClient;
 import org.apache.phoenix.index.PhoenixIndexCodec;
@@ -87,7 +86,6 @@ import org.apache.phoenix.util.LogUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.SQLCloseable;
 import org.apache.phoenix.util.SQLCloseables;
-import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.ServerUtil;
 import org.apache.phoenix.util.TransactionUtil;
 import org.slf4j.Logger;
@@ -216,7 +214,7 @@ public class MutationState implements SQLCloseable {
      */
     public void commitDDLFence(PTable dataTable) throws SQLException {
         if (dataTable.isTransactional()) {
-            byte[] key = SchemaUtil.getTableKey(dataTable);
+            byte[] key = dataTable.getName().getBytes();
             boolean success = false;
             try {
                 FenceWait fenceWait = VisibilityFence.prepareWait(key, connection.getQueryServices().getTransactionSystemClient());
@@ -250,18 +248,28 @@ public class MutationState implements SQLCloseable {
      * These entries will not conflict with each other, but they will conflict with a
      * DDL operation of creating an index. See {@link #addDMLFence(PTable)} and TEPHRA-157
      * for more information.
-     * @param dataTable the table which is doing DML
+     * @param table the table which is doing DML
      * @throws SQLException
      */
-    public void addDMLFence(PTable dataTable) throws SQLException {
+    private void addDMLFence(PTable table) throws SQLException {
+        if (table.getType() == PTableType.INDEX || !table.isTransactional()) {
+            return;
+        }
+        byte[] logicalKey = table.getName().getBytes();
+        TransactionAware logicalTxAware = VisibilityFence.create(logicalKey);
         if (this.txContext == null) {
-            throw new SQLExceptionInfo.Builder(SQLExceptionCode.NULL_TRANSACTION_CONTEXT).build().buildException();
+            this.txAwares.add(logicalTxAware);
+        } else {
+            this.txContext.addTransactionAware(logicalTxAware);
         }
-        byte[] logicalKey = SchemaUtil.getTableKey(dataTable);
-        this.txContext.addTransactionAware(VisibilityFence.create(logicalKey));
-        byte[] physicalKey = dataTable.getPhysicalName().getBytes();
+        byte[] physicalKey = table.getPhysicalName().getBytes();
         if (Bytes.compareTo(physicalKey, logicalKey) != 0) {
-            this.txContext.addTransactionAware(VisibilityFence.create(physicalKey));
+            TransactionAware physicalTxAware = VisibilityFence.create(physicalKey);
+            if (this.txContext == null) {
+                this.txAwares.add(physicalTxAware);
+            } else {
+                this.txContext.addTransactionAware(physicalTxAware);
+            }
         }
     }
     
@@ -878,6 +886,7 @@ public class MutationState implements SQLCloseable {
                 // Track tables to which we've sent uncommitted data
                 if (isTransactional = table.isTransactional()) {
                     txTableRefs.add(tableRef);
+                    addDMLFence(table);
                     uncommittedPhysicalNames.add(table.getPhysicalName().getString());
                 }
                 boolean isDataTable = true;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/424083d9/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
index b54ccd5..7e8969b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
@@ -331,7 +331,6 @@ public class PhoenixStatement implements Statement, SQLCloseable {
                                 MutationPlan plan = stmt.compilePlan(PhoenixStatement.this, Sequence.ValueOp.VALIDATE_SEQUENCE);
                                 if (plan.getTargetRef() != null && plan.getTargetRef().getTable() != null && plan.getTargetRef().getTable().isTransactional()) {
                                     state.startTransaction();
-                                    state.addDMLFence(plan.getTargetRef().getTable());
                                 }
                                 Iterator<TableRef> tableRefs = plan.getSourceRefs().iterator();
                                 state.sendUncommitted(tableRefs);


[36/50] [abbrv] phoenix git commit: PHOENIX-1726 Phoenix connection initialization makes multiple hbase connections

Posted by ma...@apache.org.
PHOENIX-1726 Phoenix connection initialization makes multiple hbase connections


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

Branch: refs/heads/calcite
Commit: bd06dfb483c5ece0069c43d120ecccedfbab3a3f
Parents: 5176ed6
Author: Samarth <sa...@salesforce.com>
Authored: Tue Jan 26 10:08:39 2016 -0800
Committer: Samarth <sa...@salesforce.com>
Committed: Tue Jan 26 10:08:39 2016 -0800

----------------------------------------------------------------------
 .../query/ConnectionQueryServicesImpl.java      | 82 ++++----------------
 1 file changed, 13 insertions(+), 69 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/bd06dfb4/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 cf21452..8eba40b 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
@@ -962,16 +962,14 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
      * @throws SQLException
      */
     private HTableDescriptor ensureTableCreated(byte[] tableName, PTableType tableType , Map<String,Object> props, List<Pair<byte[],Map<String,Object>>> families, byte[][] splits, boolean modifyExistingMetaData) throws SQLException {
-        HBaseAdmin admin = null;
         SQLException sqlE = null;
         HTableDescriptor existingDesc = null;
         boolean isMetaTable = SchemaUtil.isMetaTable(tableName);
         boolean tableExist = true;
-        try {
+        try (HBaseAdmin admin = getAdmin()) {
             final String quorum = ZKConfig.getZKQuorumServersString(config);
             final String znode = this.props.get(HConstants.ZOOKEEPER_ZNODE_PARENT);
             logger.debug("Found quorum: " + quorum + ":" + znode);
-            admin = new HBaseAdmin(config);
             try {
                 existingDesc = admin.getTableDescriptor(tableName);
             } catch (org.apache.hadoop.hbase.TableNotFoundException e) {
@@ -1065,28 +1063,16 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         } catch (TimeoutException e) {
             sqlE = new SQLExceptionInfo.Builder(SQLExceptionCode.OPERATION_TIMED_OUT).setRootCause(e.getCause() != null ? e.getCause() : e).build().buildException();
         } finally {
-            try {
-                if (admin != null) {
-                    admin.close();
-                }
-            } catch (IOException e) {
-                if (sqlE == null) {
-                    sqlE = ServerUtil.parseServerException(e);
-                } else {
-                    sqlE.setNextException(ServerUtil.parseServerException(e));
-                }
-            } finally {
-                if (sqlE != null) {
-                    throw sqlE;
-                }
+            if (sqlE != null) {
+                throw sqlE;
             }
         }
         return null; // will never make it here
     }
 
     private void modifyTable(byte[] tableName, HTableDescriptor newDesc, boolean shouldPoll) throws IOException,
-    InterruptedException, TimeoutException {
-    	try (HBaseAdmin admin = new HBaseAdmin(config)) {
+    InterruptedException, TimeoutException, SQLException {
+        try (HBaseAdmin admin = getAdmin()) {
     		if (!allowOnlineTableSchemaUpdate()) {
     			admin.disableTable(tableName);
     			admin.modifyTable(tableName, newDesc);
@@ -1308,10 +1294,8 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     private boolean ensureViewIndexTableDropped(byte[] physicalTableName, long timestamp) throws SQLException {
         byte[] physicalIndexName = MetaDataUtil.getViewIndexPhysicalName(physicalTableName);
         HTableDescriptor desc = null;
-        HBaseAdmin admin = null;
         boolean wasDeleted = false;
-        try {
-            admin = new HBaseAdmin(config);
+        try (HBaseAdmin admin = getAdmin()) {
             try {
                 desc = admin.getTableDescriptor(physicalIndexName);
                 if (Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(desc.getValue(MetaDataUtil.IS_VIEW_INDEX_TABLE_PROP_BYTES)))) {
@@ -1330,12 +1314,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             }
         } catch (IOException e) {
             throw ServerUtil.parseServerException(e);
-        } finally {
-            try {
-                if (admin != null) admin.close();
-            } catch (IOException e) {
-                logger.warn("",e);
-            }
         }
         return wasDeleted;
     }
@@ -1343,10 +1321,8 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     private boolean ensureLocalIndexTableDropped(byte[] physicalTableName, long timestamp) throws SQLException {
         byte[] physicalIndexName = MetaDataUtil.getLocalIndexPhysicalName(physicalTableName);
         HTableDescriptor desc = null;
-        HBaseAdmin admin = null;
         boolean wasDeleted = false;
-        try {
-            admin = new HBaseAdmin(config);
+        try (HBaseAdmin admin = getAdmin()) {
             try {
                 desc = admin.getTableDescriptor(physicalIndexName);
                 if (Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(desc.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES)))) {
@@ -1365,12 +1341,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             }
         } catch (IOException e) {
             throw ServerUtil.parseServerException(e);
-        } finally {
-            try {
-                if (admin != null) admin.close();
-            } catch (IOException e) {
-                logger.warn("",e);
-            }
         }
         return wasDeleted;
     }
@@ -1574,10 +1544,8 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     }
 
     private void dropTables(final List<byte[]> tableNamesToDelete) throws SQLException {
-        HBaseAdmin admin = null;
         SQLException sqlE = null;
-        try{
-            admin = new HBaseAdmin(config);
+        try (HBaseAdmin admin = getAdmin()) {
             if (tableNamesToDelete != null){
                 for ( byte[] tableName : tableNamesToDelete ) {
                     if ( admin.tableExists(tableName) ) {
@@ -1591,20 +1559,8 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         } catch (IOException e) {
             sqlE = ServerUtil.parseServerException(e);
         } finally {
-            try {
-                if (admin != null) {
-                    admin.close();
-                }
-            } catch (IOException e) {
-                if (sqlE == null) {
-                    sqlE = ServerUtil.parseServerException(e);
-                } else {
-                    sqlE.setNextException(ServerUtil.parseServerException(e));
-                }
-            } finally {
-                if (sqlE != null) {
-                    throw sqlE;
-                }
+            if (sqlE != null) {
+                throw sqlE;
             }
         }
     }
@@ -1790,10 +1746,8 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     }
     private void updateDescriptorForTx(PTable table, Map<String, Object> tableProps, HTableDescriptor tableDescriptor,
             String txValue, Set<HTableDescriptor> descriptorsToUpdate, Set<HTableDescriptor> origDescriptors) throws SQLException {
-        HBaseAdmin admin = null;
         byte[] physicalTableName = table.getPhysicalName().getBytes();
-        try {
-            admin = new HBaseAdmin(config);
+        try (HBaseAdmin admin = getAdmin()) {
             setTransactional(tableDescriptor, table.getType(), txValue, tableProps);
             Map<String, Object> indexTableProps;
             if (txValue == null) {
@@ -1848,12 +1802,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             }
         } catch (IOException e) {
             throw ServerUtil.parseServerException(e);
-        } finally {
-            try {
-                if (admin != null) admin.close();
-            } catch (IOException e) {
-                logger.warn("Could not close admin",e);
-            }
         }
     }
     private void setSharedIndexMaxVersion(PTable table, HTableDescriptor tableDescriptor,
@@ -2336,9 +2284,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                 if(currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_3_0) {
                                     // We know that we always need to add the STORE_NULLS column for 4.3 release
                                     columnsToAdd += "," + PhoenixDatabaseMetaData.STORE_NULLS + " " + PBoolean.INSTANCE.getSqlTypeName();
-                                    HBaseAdmin admin = null;
-                                    try {
-                                        admin = getAdmin();
+                                    try (HBaseAdmin admin = getAdmin()) {
                                         HTableDescriptor[] localIndexTables = admin.listTables(MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX+".*");
                                         for (HTableDescriptor table : localIndexTables) {
                                             if (table.getValue(MetaDataUtil.PARENT_TABLE_KEY) == null
@@ -2354,8 +2300,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                                 admin.enableTable(table.getTableName());
                                             }
                                         }
-                                    } finally {
-                                        if (admin != null) admin.close();
                                     }
                                 }
 
@@ -2732,7 +2676,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     @Override
     public HBaseAdmin getAdmin() throws SQLException {
         try {
-            return new HBaseAdmin(config);
+            return new HBaseAdmin(connection);
         } catch (IOException e) {
             throw new PhoenixIOException(e);
         }


[12/50] [abbrv] phoenix git commit: PHOENIX-2608 Incompatibility between Jackson1 version shipped with Phoenix, Yarn (Ted Yu)

Posted by ma...@apache.org.
PHOENIX-2608 Incompatibility between Jackson1 version shipped with Phoenix, Yarn (Ted Yu)


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

Branch: refs/heads/calcite
Commit: b63ca5fbdf60e6dd9a2d0d054c3be058e3b6b5c5
Parents: 588f449
Author: Nick Dimiduk <nd...@apache.org>
Authored: Tue Jan 19 20:20:36 2016 -0800
Committer: Nick Dimiduk <nd...@apache.org>
Committed: Tue Jan 19 20:51:10 2016 -0800

----------------------------------------------------------------------
 pom.xml | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/b63ca5fb/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 72551a4..2c47ca2 100644
--- a/pom.xml
+++ b/pom.xml
@@ -87,7 +87,7 @@
     <commons-cli.version>1.2</commons-cli.version>
     <hadoop.version>2.5.1</hadoop.version>
     <pig.version>0.13.0</pig.version>
-    <jackson.version>1.8.8</jackson.version>
+    <jackson.version>1.9.2</jackson.version>
     <antlr.version>3.5</antlr.version>
     <log4j.version>1.2.17</log4j.version>
     <slf4j.version>1.6.4</slf4j.version>
@@ -99,7 +99,6 @@
     <commons-csv.version>1.0</commons-csv.version>
     <sqlline.version>1.1.8</sqlline.version>
     <guava.version>13.0.1</guava.version>
-    <jackson.version>1.8.8</jackson.version>
     <flume.version>1.4.0</flume.version>
     <findbugs.version>1.3.2</findbugs.version>
     <jline.version>2.11</jline.version>