You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ja...@apache.org on 2015/08/12 03:39:53 UTC

[01/12] phoenix git commit: Revert "PHOENIX-1791. Adding ability for Pherf Write Workloads to write to a multi-tenant view."

Repository: phoenix
Updated Branches:
  refs/heads/4.5-HBase-0.98 e9d8c01c8 -> 71da8a021


Revert "PHOENIX-1791. Adding ability for Pherf Write Workloads to write to a multi-tenant view."

This reverts commit e9d8c01c8141a2f402bd8466ebb71ed5418a1638.


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

Branch: refs/heads/4.5-HBase-0.98
Commit: 43cd3ae402d6c0dd16da73d96f1e4e47932f497e
Parents: e9d8c01
Author: James Taylor <jt...@salesforce.com>
Authored: Tue Aug 11 18:12:43 2015 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Tue Aug 11 18:36:51 2015 -0700

----------------------------------------------------------------------
 .../org/apache/phoenix/pherf/DataIngestIT.java  | 74 +++-----------------
 .../phoenix/pherf/configuration/Scenario.java   | 25 ++-----
 .../pherf/configuration/XMLConfigParser.java    | 23 +++---
 .../apache/phoenix/pherf/util/PhoenixUtil.java  | 28 +++-----
 .../phoenix/pherf/workload/WriteWorkload.java   | 36 +++++-----
 .../datamodel/test_schema_mt_table.sql          | 31 --------
 .../resources/datamodel/test_schema_mt_view.sql |  1 -
 .../test/resources/scenario/test_scenario.xml   | 22 ------
 8 files changed, 53 insertions(+), 187 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/43cd3ae4/phoenix-pherf/src/it/java/org/apache/phoenix/pherf/DataIngestIT.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/it/java/org/apache/phoenix/pherf/DataIngestIT.java b/phoenix-pherf/src/it/java/org/apache/phoenix/pherf/DataIngestIT.java
index 2b9d309..8a340b3 100644
--- a/phoenix-pherf/src/it/java/org/apache/phoenix/pherf/DataIngestIT.java
+++ b/phoenix-pherf/src/it/java/org/apache/phoenix/pherf/DataIngestIT.java
@@ -18,19 +18,8 @@
 
 package org.apache.phoenix.pherf;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.sql.Connection;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
+import com.jcabi.jdbc.JdbcSession;
+import com.jcabi.jdbc.Outcome;
 import org.apache.phoenix.pherf.configuration.Column;
 import org.apache.phoenix.pherf.configuration.DataModel;
 import org.apache.phoenix.pherf.configuration.DataTypeMapping;
@@ -44,8 +33,15 @@ import org.apache.phoenix.pherf.workload.WriteWorkload;
 import org.junit.Before;
 import org.junit.Test;
 
-import com.jcabi.jdbc.JdbcSession;
-import com.jcabi.jdbc.Outcome;
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.*;
 
 public class DataIngestIT extends ResultBaseTestIT {
 
@@ -148,52 +144,4 @@ public class DataIngestIT extends ResultBaseTestIT {
             fail("Failed to load data. An exception was thrown: " + e.getMessage());
         }
     }
-
-
-    @Test
-    /**
-     * Validates that Pherf can write data to a Multi-Tenant View in addition to 
-     * standard Phoenix tables.
-     */
-    public void testMultiTenantViewWriteWorkload() throws Exception {
-        // Arrange
-        Scenario scenario = parser.getScenarioByName("testMTWriteScenario");
-        WorkloadExecutor executor = new WorkloadExecutor();
-        executor.add(new WriteWorkload(util, parser, scenario));
-        
-        // Act
-        try {
-            // Wait for data to load up.
-            executor.get();
-            executor.shutdown();
-        } catch (Exception e) {
-            fail("Failed to load data. An exception was thrown: " + e.getMessage());
-        }
-
-        // Assert
-        assertExpectedNumberOfRecordsWritten(scenario);
-
-    }
-
-    private void assertExpectedNumberOfRecordsWritten(Scenario scenario) throws Exception,
-            SQLException {
-        Connection connection = util.getConnection(scenario.getTenantId());
-        String sql = "select count(*) from " + scenario.getTableName();
-        Integer count = new JdbcSession(connection).sql(sql).select(new Outcome<Integer>() {
-            @Override public Integer handle(ResultSet resultSet, Statement statement)
-                    throws SQLException {
-                while (resultSet.next()) {
-                    return resultSet.getInt(1);
-                }
-                return null;
-            }
-        });
-        assertNotNull("Could not retrieve count. " + count);
-        assertTrue("Could not query any rows for in " + scenario.getTableName(), count > 0);
-        assertEquals("Expected 100 rows to have been inserted", scenario.getRowCount(), count.intValue());
-    }
-    
-
-
-
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/43cd3ae4/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/Scenario.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/Scenario.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/Scenario.java
index 6c949d8..7de96cc 100644
--- a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/Scenario.java
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/Scenario.java
@@ -18,16 +18,15 @@
 
 package org.apache.phoenix.pherf.configuration;
 
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.phoenix.pherf.util.PhoenixUtil;
 
 import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlRootElement;
-
-import org.apache.commons.lang.builder.HashCodeBuilder;
-import org.apache.phoenix.pherf.util.PhoenixUtil;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
 
 @XmlRootElement(namespace = "org.apache.phoenix.pherf.configuration.DataModel")
 public class Scenario {
@@ -38,7 +37,6 @@ public class Scenario {
     private List<QuerySet> querySet = new ArrayList<>();
     private WriteParams writeParams;
     private String name;
-    private String tenantId;
 
     public Scenario() {
         writeParams = new WriteParams();
@@ -164,19 +162,6 @@ public class Scenario {
     public void setName(String name) {
         this.name = name;
     }
-    
-    /**
-     * Tenant Id used by connection of this query
-     * @return
-     */
-    @XmlAttribute
-    public String getTenantId() {
-        return tenantId;
-    }
-
-    public void setTenantId(String tenantId) {
-        this.tenantId = tenantId;
-    }
 
     public WriteParams getWriteParams() {
         return writeParams;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/43cd3ae4/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/XMLConfigParser.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/XMLConfigParser.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/XMLConfigParser.java
index 93dc94c..393fa7e 100644
--- a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/XMLConfigParser.java
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/XMLConfigParser.java
@@ -18,6 +18,16 @@
 
 package org.apache.phoenix.pherf.configuration;
 
+import org.apache.phoenix.pherf.PherfConstants;
+import org.apache.phoenix.pherf.exception.FileLoaderException;
+import org.apache.phoenix.pherf.util.ResourceList;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.JAXBException;
+import javax.xml.bind.Marshaller;
+import javax.xml.bind.Unmarshaller;
 import java.io.OutputStream;
 import java.nio.file.Path;
 import java.util.ArrayList;
@@ -25,17 +35,6 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 
-import javax.xml.bind.JAXBContext;
-import javax.xml.bind.JAXBException;
-import javax.xml.bind.Marshaller;
-import javax.xml.bind.Unmarshaller;
-
-import org.apache.phoenix.pherf.PherfConstants;
-import org.apache.phoenix.pherf.exception.FileLoaderException;
-import org.apache.phoenix.pherf.util.ResourceList;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 public class XMLConfigParser {
 
     private static final Logger logger = LoggerFactory.getLogger(XMLConfigParser.class);
@@ -135,8 +134,6 @@ public class XMLConfigParser {
         if (fullTableName.contains(".")) {
             ret = fullTableName.substring(fullTableName.indexOf(".") + 1, fullTableName.length());
         }
-        // Remove any quotes that may be needed for multi-tenant tables
-        ret = ret.replaceAll("\"", "");
         return ret;
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/43cd3ae4/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/util/PhoenixUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/util/PhoenixUtil.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/util/PhoenixUtil.java
index 48d3888..0156149 100644
--- a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/util/PhoenixUtil.java
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/util/PhoenixUtil.java
@@ -18,26 +18,20 @@
 
 package org.apache.phoenix.pherf.util;
 
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_NAME;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SCHEM;
-
-import java.sql.Connection;
-import java.sql.DatabaseMetaData;
-import java.sql.DriverManager;
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Properties;
-
 import org.apache.phoenix.pherf.PherfConstants;
 import org.apache.phoenix.pherf.configuration.Column;
 import org.apache.phoenix.pherf.configuration.DataTypeMapping;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.sql.*;
+import java.util.*;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.phoenix.pherf.configuration.Query;
 import org.apache.phoenix.pherf.configuration.QuerySet;
-import org.apache.phoenix.pherf.configuration.Scenario;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -88,11 +82,11 @@ public class PhoenixUtil {
         return DriverManager.getConnection(url, props);
     }
 
-    public boolean executeStatement(String sql, Scenario scenario) throws Exception {
+    public boolean executeStatement(String sql) throws Exception {
         Connection connection = null;
         boolean result = false;
         try {
-            connection = getConnection(scenario.getTenantId());
+            connection = getConnection();
             result = executeStatement(sql, connection);
         } finally {
             if (connection != null) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/43cd3ae4/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/WriteWorkload.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/WriteWorkload.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/WriteWorkload.java
index 4e7fbfc..305521b 100644
--- a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/WriteWorkload.java
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/WriteWorkload.java
@@ -18,21 +18,6 @@
 
 package org.apache.phoenix.pherf.workload;
 
-import java.math.BigDecimal;
-import java.sql.Connection;
-import java.sql.Date;
-import java.sql.PreparedStatement;
-import java.sql.SQLException;
-import java.sql.Types;
-import java.text.SimpleDateFormat;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Properties;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-
 import org.apache.phoenix.pherf.PherfConstants;
 import org.apache.phoenix.pherf.configuration.Column;
 import org.apache.phoenix.pherf.configuration.Scenario;
@@ -49,6 +34,17 @@ import org.apache.phoenix.pherf.util.RowCalculator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.math.BigDecimal;
+import java.sql.*;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
 public class WriteWorkload implements Workload {
     private static final Logger logger = LoggerFactory.getLogger(WriteWorkload.class);
     private final PhoenixUtil pUtil;
@@ -162,7 +158,7 @@ public class WriteWorkload implements Workload {
         waitForBatches(dataLoadTimeSummary, scenario, start, writeBatches);
 
         // always update stats for Phoenix base tables
-        updatePhoenixStats(scenario.getTableName(), scenario);
+        updatePhoenixStats(scenario.getTableName());
     }
 
     private List<Future> getBatches(DataLoadThreadTime dataLoadThreadTime, Scenario scenario)
@@ -176,7 +172,7 @@ public class WriteWorkload implements Workload {
             List<Column>
                     phxMetaCols =
                     pUtil.getColumnsFromPhoenix(scenario.getSchemaName(),
-                            scenario.getTableNameWithoutSchemaName(), pUtil.getConnection(scenario.getTenantId()));
+                            scenario.getTableNameWithoutSchemaName(), pUtil.getConnection());
             int threadRowCount = rowCalculator.getNext();
             logger.info(
                     "Kick off thread (#" + i + ")for upsert with (" + threadRowCount + ") rows.");
@@ -219,9 +215,9 @@ public class WriteWorkload implements Workload {
      * @param tableName
      * @throws Exception
      */
-    public void updatePhoenixStats(String tableName, Scenario scenario) throws Exception {
+    public void updatePhoenixStats(String tableName) throws Exception {
         logger.info("Updating stats for " + tableName);
-        pUtil.executeStatement("UPDATE STATISTICS " + tableName, scenario);
+        pUtil.executeStatement("UPDATE STATISTICS " + tableName);
     }
 
     public Future<Info> upsertData(final Scenario scenario, final List<Column> columns,
@@ -234,7 +230,7 @@ public class WriteWorkload implements Workload {
                 SimpleDateFormat simpleDateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
                 Connection connection = null;
                 try {
-                    connection = pUtil.getConnection(scenario.getTenantId());
+                    connection = pUtil.getConnection();
                     long logStartTime = System.currentTimeMillis();
                     long
                             maxDuration =

http://git-wip-us.apache.org/repos/asf/phoenix/blob/43cd3ae4/phoenix-pherf/src/test/resources/datamodel/test_schema_mt_table.sql
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/test/resources/datamodel/test_schema_mt_table.sql b/phoenix-pherf/src/test/resources/datamodel/test_schema_mt_table.sql
deleted file mode 100644
index b6791bf..0000000
--- a/phoenix-pherf/src/test/resources/datamodel/test_schema_mt_table.sql
+++ /dev/null
@@ -1,31 +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.
-*/
-CREATE TABLE IF NOT EXISTS PHERF.TEST_MULTI_TENANT_TABLE (
-    TENANT_ID CHAR(15) NOT NULL,
-    IDENTIFIER CHAR(3) NOT NULL,
-    ID CHAR(15) NOT NULL,
-    CREATED_DATE DATE,
-    FIELD VARCHAR,
-    SOME_INT INTEGER
-    CONSTRAINT PK PRIMARY KEY
-    (
-        TENANT_ID,
-        IDENTIFIER,
-        ID
-    )
-) VERSIONS=1,MULTI_TENANT=true

http://git-wip-us.apache.org/repos/asf/phoenix/blob/43cd3ae4/phoenix-pherf/src/test/resources/datamodel/test_schema_mt_view.sql
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/test/resources/datamodel/test_schema_mt_view.sql b/phoenix-pherf/src/test/resources/datamodel/test_schema_mt_view.sql
deleted file mode 100644
index 5f5d7ec..0000000
--- a/phoenix-pherf/src/test/resources/datamodel/test_schema_mt_view.sql
+++ /dev/null
@@ -1 +0,0 @@
-CREATE VIEW IF NOT EXISTS PHERF.TEST_VIEW (field1 VARCHAR, field2 VARCHAR) AS SELECT * FROM PHERF.TEST_MULTI_TENANT_TABLE
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/43cd3ae4/phoenix-pherf/src/test/resources/scenario/test_scenario.xml
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/test/resources/scenario/test_scenario.xml b/phoenix-pherf/src/test/resources/scenario/test_scenario.xml
index b5fe564..fddf022 100644
--- a/phoenix-pherf/src/test/resources/scenario/test_scenario.xml
+++ b/phoenix-pherf/src/test/resources/scenario/test_scenario.xml
@@ -127,25 +127,6 @@
             <name>NEWVAL_STRING</name>
             <prefix>TSTPRFX</prefix>
         </column>
-        <column>
-            <type>CHAR</type>
-            <length>3</length>
-            <userDefined>true</userDefined>
-            <dataSequence>LIST</dataSequence>
-            <name>IDENTIFIER</name>
-            <valuelist>
-                <!-- Distributes according to specified values. These must total 100 -->
-                <datavalue distribution="60">
-                    <value>ABC</value>
-                </datavalue>
-                <datavalue distribution="20">
-                    <value>XYZ</value>
-                </datavalue>
-                <datavalue distribution="20">
-                    <value>LMN</value>
-                </datavalue>
-            </valuelist>            
-        </column>        
     </datamapping>
     <scenarios>
         <scenario tableName="PHERF.TEST_TABLE" rowCount="100" name="testScenarioRW">
@@ -220,8 +201,5 @@
                 <query id="q4" statement="select sum(SOME_INT) from PHERF.TEST_TABLE"/>
             </querySet>
         </scenario>
-        <!-- Test writing to a Multi-tenant View -->
-        <scenario tableName="PHERF.TEST_VIEW" tenantId="abcdefghijklmno" rowCount="100" name="testMTWriteScenario">
-        </scenario>
     </scenarios>
 </datamodel>
\ No newline at end of file


[02/12] phoenix git commit: PHOENIX-2171 DOUBLE and FLOAT DESC are stored as ASC

Posted by ja...@apache.org.
PHOENIX-2171 DOUBLE and FLOAT DESC are stored as ASC


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

Branch: refs/heads/4.5-HBase-0.98
Commit: 71da8a021279e69de1928605fd03c6e8bcdc1ed0
Parents: b642514
Author: James Taylor <jt...@salesforce.com>
Authored: Tue Aug 11 01:59:23 2015 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Tue Aug 11 18:36:52 2015 -0700

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/SortOrderIT.java |  6 ++--
 .../UngroupedAggregateRegionObserver.java       |  6 ++++
 .../org/apache/phoenix/schema/PTableImpl.java   | 10 +++++-
 .../apache/phoenix/schema/types/PDataType.java  |  4 +--
 .../apache/phoenix/schema/types/PDouble.java    | 18 +++++++----
 .../org/apache/phoenix/schema/types/PFloat.java | 18 ++++++-----
 .../org/apache/phoenix/util/UpgradeUtil.java    |  5 +++
 .../phoenix/schema/types/PDataTypeTest.java     | 33 ++++++++++++++++++++
 8 files changed, 82 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/71da8a02/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderIT.java
index 9228ab5..fdbd26d 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderIT.java
@@ -41,6 +41,8 @@ import org.apache.commons.lang.ArrayUtils;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PDecimal;
+import org.apache.phoenix.schema.types.PDouble;
+import org.apache.phoenix.schema.types.PFloat;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.junit.Assert;
 import org.junit.Test;
@@ -404,7 +406,7 @@ public class SortOrderIT extends BaseHBaseManagedTimeIT {
     public void testNonPKCompare() throws Exception {
         List<Integer> expectedResults = Lists.newArrayList(2,3,4);
         Integer[] saltBuckets = new Integer[] {null,3};
-        PDataType[] dataTypes = new PDataType[] {PDecimal.INSTANCE};
+        PDataType[] dataTypes = new PDataType[] {PDecimal.INSTANCE, PDouble.INSTANCE, PFloat.INSTANCE};
         for (Integer saltBucket : saltBuckets) {
             for (PDataType dataType : dataTypes) {
                 for (SortOrder sortOrder : SortOrder.values()) {
@@ -420,7 +422,7 @@ public class SortOrderIT extends BaseHBaseManagedTimeIT {
         List<Integer> rExpectedResults = new ArrayList<>(expectedResults);
         Collections.reverse(rExpectedResults);
         Integer[] saltBuckets = new Integer[] {null,3};
-        PDataType[] dataTypes = new PDataType[] {PDecimal.INSTANCE};
+        PDataType[] dataTypes = new PDataType[] {PDecimal.INSTANCE, PDouble.INSTANCE, PFloat.INSTANCE};
         for (Integer saltBucket : saltBuckets) {
             for (PDataType dataType : dataTypes) {
                 for (SortOrder sortOrder : SortOrder.values()) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/71da8a02/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 571d0d1..7316bb1 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
@@ -88,6 +88,8 @@ import org.apache.phoenix.schema.tuple.MultiKeyValueTuple;
 import org.apache.phoenix.schema.types.PBinary;
 import org.apache.phoenix.schema.types.PChar;
 import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PDouble;
+import org.apache.phoenix.schema.types.PFloat;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.KeyValueUtil;
@@ -312,6 +314,10 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver{
                                                 len--;
                                             }
                                             ptr.set(ptr.get(), ptr.getOffset(), len);
+                                        // Special case for re-writing DESC FLOAT and DOUBLE, as they're not inverted like they should be (PHOENIX-2171)
+                                        } else if (field.getDataType() == PFloat.INSTANCE || field.getDataType() == PDouble.INSTANCE) {
+                                            byte[] invertedBytes = SortOrder.invert(ptr.get(), ptr.getOffset(), ptr.getLength());
+                                            ptr.set(invertedBytes);
                                         }
                                     } else if (field.getDataType() == PBinary.INSTANCE) {
                                         // Remove trailing space characters so that the setValues call below will replace them

http://git-wip-us.apache.org/repos/asf/phoenix/blob/71da8a02/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 610bd83..407ee90 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
@@ -57,6 +57,8 @@ import org.apache.phoenix.schema.stats.PTableStatsImpl;
 import org.apache.phoenix.schema.types.PBinary;
 import org.apache.phoenix.schema.types.PChar;
 import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PDouble;
+import org.apache.phoenix.schema.types.PFloat;
 import org.apache.phoenix.schema.types.PVarchar;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.SchemaUtil;
@@ -407,7 +409,13 @@ public class PTableImpl implements PTable {
         for (PColumn column : allColumns) {
             PName familyName = column.getFamilyName();
             if (familyName == null) {
-                hasColumnsRequiringUpgrade |= (column.getSortOrder() == SortOrder.DESC && (!column.getDataType().isFixedWidth() || column.getDataType() == PChar.INSTANCE || column.getDataType() == PBinary.INSTANCE))
+                hasColumnsRequiringUpgrade |= 
+                        ( column.getSortOrder() == SortOrder.DESC 
+                            && (!column.getDataType().isFixedWidth() 
+                                || column.getDataType() == PChar.INSTANCE 
+                                || column.getDataType() == PFloat.INSTANCE 
+                                || column.getDataType() == PDouble.INSTANCE 
+                                || column.getDataType() == PBinary.INSTANCE) )
                         || (column.getSortOrder() == SortOrder.ASC && column.getDataType() == PBinary.INSTANCE && column.getMaxLength() != null && column.getMaxLength() > 1);
             	pkColumns.add(column);
             }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/71da8a02/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDataType.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDataType.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDataType.java
index d79de60..5d8852c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDataType.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDataType.java
@@ -66,16 +66,15 @@ public abstract class PDataType<T> implements DataType<T>, Comparable<PDataType<
         this.ordinal = ordinal;
     }
 
-    @Deprecated
     public static PDataType[] values() {
         return PDataTypeFactory.getInstance().getOrderedTypes();
     }
 
-    @Deprecated
     public int ordinal() {
         return ordinal;
     }
 
+    @SuppressWarnings("unchecked")
     @Override
     public Class<T> encodedClass() {
         return getJavaClass();
@@ -942,6 +941,7 @@ public abstract class PDataType<T> implements DataType<T>, Comparable<PDataType<
     public abstract Object toObject(byte[] bytes, int offset, int length, PDataType actualType, SortOrder sortOrder,
             Integer maxLength, Integer scale);
 
+    @SuppressWarnings("unchecked")
     @Override
     public T decode(PositionedByteRange pbr) {
         // default implementation based on existing PDataType methods.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/71da8a02/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDouble.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDouble.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDouble.java
index d11aedf..95a526e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDouble.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDouble.java
@@ -236,15 +236,21 @@ public class PDouble extends PRealNumber<Double> {
     }
 
     @Override
-    public double decodeDouble(byte[] b, int o, SortOrder sortOrder) {
+    public double decodeDouble(byte[] bytes, int o, SortOrder sortOrder) {
       Preconditions.checkNotNull(sortOrder);
-      checkForSufficientLength(b, o, Bytes.SIZEOF_LONG);
+      checkForSufficientLength(bytes, o, Bytes.SIZEOF_LONG);
+      long l;
       if (sortOrder == SortOrder.DESC) {
-        for (int i = o; i < Bytes.SIZEOF_LONG; i++) {
-          b[i] = (byte) (b[i] ^ 0xff);
-        }
+          // Copied from Bytes.toLong(), but without using the toLongUnsafe
+          // TODO: would it be possible to use the toLongUnsafe?
+          l = 0;
+          for(int i = o; i < o + Bytes.SIZEOF_LONG; i++) {
+            l <<= 8;
+            l ^= (bytes[i] ^ 0xff) & 0xFF;
+          }
+      } else {
+          l = Bytes.toLong(bytes, o);
       }
-      long l = Bytes.toLong(b, o);
       l--;
       l ^= (~l >> Long.SIZE - 1) | Long.MIN_VALUE;
       return Double.longBitsToDouble(l);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/71da8a02/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PFloat.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PFloat.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PFloat.java
index 67b7d9a..75f8efa 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PFloat.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PFloat.java
@@ -243,15 +243,19 @@ public class PFloat extends PRealNumber<Float> {
     public float decodeFloat(byte[] b, int o, SortOrder sortOrder) {
       Preconditions.checkNotNull(sortOrder);
       checkForSufficientLength(b, o, Bytes.SIZEOF_INT);
+      int value;
       if (sortOrder == SortOrder.DESC) {
-        for (int i = o; i < Bytes.SIZEOF_INT; i++) {
-          b[i] = (byte) (b[i] ^ 0xff);
-        }
+          value = 0;
+          for(int i = o; i < (o + Bytes.SIZEOF_INT); i++) {
+            value <<= 8;
+            value ^= (b[i] ^ 0xff) & 0xFF;
+          }
+      } else {
+          value = Bytes.toInt(b, o);
       }
-      int i = Bytes.toInt(b, o);
-      i--;
-      i ^= (~i >> Integer.SIZE - 1) | Integer.MIN_VALUE;
-      return Float.intBitsToFloat(i);
+      value--;
+      value ^= (~value >> Integer.SIZE - 1) | Integer.MIN_VALUE;
+      return Float.intBitsToFloat(value);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/71da8a02/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 b303b09..f2b4516 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
@@ -84,6 +84,8 @@ import org.apache.phoenix.schema.types.PBoolean;
 import org.apache.phoenix.schema.types.PChar;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PDecimal;
+import org.apache.phoenix.schema.types.PDouble;
+import org.apache.phoenix.schema.types.PFloat;
 import org.apache.phoenix.schema.types.PInteger;
 import org.apache.phoenix.schema.types.PLong;
 import org.apache.phoenix.schema.types.PVarbinary;
@@ -884,12 +886,15 @@ public class UpgradeUtil {
     // Return all types that are descending and either:
     // 1) variable length, which includes all array types (PHOENIX-2067)
     // 2) fixed length with padding (PHOENIX-2120)
+    // 3) float and double (PHOENIX-2171)
     // We exclude VARBINARY as we no longer support DESC for it.
     private static String getAffectedDataTypes() {
         StringBuilder buf = new StringBuilder("(" 
                 + PVarchar.INSTANCE.getSqlType() + "," +
                 + PChar.INSTANCE.getSqlType() + "," +
                 + PBinary.INSTANCE.getSqlType() + "," +
+                + PFloat.INSTANCE.getSqlType() + "," +
+                + PDouble.INSTANCE.getSqlType() + "," +
                 + PDecimal.INSTANCE.getSqlType() + ","
                 );
         for (PDataType type : PDataType.values()) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/71da8a02/phoenix-core/src/test/java/org/apache/phoenix/schema/types/PDataTypeTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/schema/types/PDataTypeTest.java b/phoenix-core/src/test/java/org/apache/phoenix/schema/types/PDataTypeTest.java
index 7ab9093..5657c22 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/schema/types/PDataTypeTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/schema/types/PDataTypeTest.java
@@ -41,6 +41,7 @@ import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.ConstraintViolationException;
 import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.util.ScanUtil;
 import org.apache.phoenix.util.TestUtil;
 import org.junit.Test;
 
@@ -743,6 +744,38 @@ public class PDataTypeTest {
     }
     
     @Test
+    public void testDoubleComparison() {
+        testRealNumberComparison(PDouble.INSTANCE, new Double[] {0.99, 1.0, 1.001, 1.01, 2.0});
+    }
+    
+    @Test
+    public void testFloatComparison() {
+        testRealNumberComparison(PFloat.INSTANCE, new Float[] {0.99f, 1.0f, 1.001f, 1.01f, 2.0f});
+    }
+    
+    @Test
+    public void testDecimalComparison() {
+        testRealNumberComparison(PDecimal.INSTANCE, new BigDecimal[] {BigDecimal.valueOf(0.99), BigDecimal.valueOf(1.0), BigDecimal.valueOf(1.001), BigDecimal.valueOf(1.01), BigDecimal.valueOf(2.0)});
+    }
+    
+    private static void testRealNumberComparison(PDataType type, Object[] a) {
+        
+        for (SortOrder sortOrder : SortOrder.values()) {
+            int factor = (sortOrder == SortOrder.ASC ? 1 : -1);
+            byte[] prev_b = null;
+            Object prev_o = null;
+            for (Object o : a) {
+                byte[] b = type.toBytes(o, sortOrder);
+                if (prev_b != null) {
+                    assertTrue("Compare of " + o + " with " + prev_o + " " + sortOrder + " failed.", ScanUtil.getComparator(type.isFixedWidth(), sortOrder).compare(prev_b, 0, prev_b.length, b, 0, b.length) * factor < 0);
+                }
+                prev_b = b;
+                prev_o = o;
+            }
+        }
+    }
+    
+    @Test
     public void testDouble() {
         Double na = 0.005;
         byte[] b = PDouble.INSTANCE.toBytes(na);


[03/12] phoenix git commit: PHOENIX-2149 MAX Value of Sequences not honored when closing Connection between calls to NEXT VALUE FOR (Jan Fernando)

Posted by ja...@apache.org.
PHOENIX-2149 MAX Value of Sequences not honored when closing Connection between calls to NEXT VALUE FOR (Jan Fernando)


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

Branch: refs/heads/4.5-HBase-0.98
Commit: 3f03ced5392ecc324affcbdf3a51d2c4ecc18126
Parents: e78a648
Author: Thomas D'Silva <td...@salesforce.com>
Authored: Mon Jul 27 13:15:17 2015 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Tue Aug 11 18:36:52 2015 -0700

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/SequenceIT.java  | 33 ++++++++++++++++++++
 .../org/apache/phoenix/schema/Sequence.java     |  3 +-
 2 files changed, 34 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/3f03ced5/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceIT.java
index 4273022..78f8132 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceIT.java
@@ -54,6 +54,7 @@ import com.google.common.collect.Lists;
 
 public class SequenceIT extends BaseClientManagedTimeIT {
     private static final String NEXT_VAL_SQL = "SELECT NEXT VALUE FOR foo.bar FROM SYSTEM.\"SEQUENCE\"";
+    private static final String SELECT_NEXT_VALUE_SQL = "SELECT NEXT VALUE FOR %s FROM SYSTEM.\"SEQUENCE\"";
     private static final long BATCH_SIZE = 3;
    
     private Connection conn;
@@ -1147,6 +1148,38 @@ public class SequenceIT extends BaseClientManagedTimeIT {
         assertEquals(1, rs.getLong("metric_val"));
         assertFalse(rs.next());
     }
+    
+    @Test
+    /**
+     * Test to validate that the bug discovered in PHOENIX-2149 has been fixed. There was an issue
+     * whereby, when closing connections and returning sequences we were not setting the limit
+     * reached flag correctly and this was causing the max value to be ignored as the LIMIT_REACHED_FLAG
+     * value was being unset from true to false.
+     */
+    public void testNextValuesForSequenceClosingConnections() throws Exception {
+
+        // Create Sequence
+        nextConnection();
+        conn.createStatement().execute("CREATE SEQUENCE seqtest.closeconn START WITH 4990 MINVALUE 4990 MAXVALUE 5000 CACHE 10");
+        nextConnection();
+        
+        // Call NEXT VALUE FOR 1 time more than available values in the Sequence. We expected the final time
+        // to throw an error as we will have reached the max value
+        try {
+            long val = 0L;
+            for (int i = 0; i <= 11; i++) {
+                ResultSet rs = conn.createStatement().executeQuery(String.format(SELECT_NEXT_VALUE_SQL, "seqtest.closeconn"));
+                rs.next();
+                val = rs.getLong(1);
+                nextConnection();
+            }
+            fail("Expect to fail as we have arrived at the max sequence value " + val);
+        } catch (SQLException e) {
+            assertEquals(SQLExceptionCode.SEQUENCE_VAL_REACHED_MAX_VALUE.getErrorCode(),
+                e.getErrorCode());
+            assertTrue(e.getNextException() == null);
+        }
+    }
 
     private void insertEvent(long id, String userId, long val) throws SQLException {
         PreparedStatement stmt = conn.prepareStatement("UPSERT INTO events VALUES(?,?,?)");

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3f03ced5/phoenix-core/src/main/java/org/apache/phoenix/schema/Sequence.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/Sequence.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/Sequence.java
index adca5e8..a2041f7 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/Sequence.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/Sequence.java
@@ -302,8 +302,7 @@ public class Sequence {
         Map<byte[], List<Cell>> familyMap = append.getFamilyCellMap();
         familyMap.put(PhoenixDatabaseMetaData.SEQUENCE_FAMILY_BYTES, Arrays.<Cell>asList(
         		(Cell)KeyValueUtil.newKeyValue(key, PhoenixDatabaseMetaData.SEQUENCE_FAMILY_BYTES, PhoenixDatabaseMetaData.CURRENT_VALUE_BYTES, value.timestamp, PLong.INSTANCE.toBytes(value.currentValue)),
-        		// set LIMIT_REACHED flag to false since we are returning unused sequence values
-        		(Cell)KeyValueUtil.newKeyValue(key, PhoenixDatabaseMetaData.SEQUENCE_FAMILY_BYTES, PhoenixDatabaseMetaData.LIMIT_REACHED_FLAG_BYTES, value.timestamp, PDataType.FALSE_BYTES)
+        		(Cell)KeyValueUtil.newKeyValue(key, PhoenixDatabaseMetaData.SEQUENCE_FAMILY_BYTES, PhoenixDatabaseMetaData.LIMIT_REACHED_FLAG_BYTES, value.timestamp, PBoolean.INSTANCE.toBytes(value.limitReached))
                 ));
         return append;
     }


[06/12] phoenix git commit: PHOENIX 2132 Minor addendum to fix RAT warning

Posted by ja...@apache.org.
PHOENIX 2132 Minor addendum to fix RAT warning


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

Branch: refs/heads/4.5-HBase-0.98
Commit: b1ad90f05513c66aa0250e0044823063307dcf59
Parents: fd7523f
Author: Mujtaba <mu...@apache.org>
Authored: Fri Jul 24 11:55:45 2015 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Tue Aug 11 18:36:52 2015 -0700

----------------------------------------------------------------------
 phoenix-pherf/pom.xml | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/b1ad90f0/phoenix-pherf/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-pherf/pom.xml b/phoenix-pherf/pom.xml
index 6824b1f..68a8905 100644
--- a/phoenix-pherf/pom.xml
+++ b/phoenix-pherf/pom.xml
@@ -195,7 +195,8 @@
 				<artifactId>apache-rat-plugin</artifactId>
 				<configuration>
 					<excludes>
-						<exclude>RESULTS/**</exclude>
+						<exclude>*/RESULTS/**</exclude>
+						<exclude>README.md</exclude>
 					</excludes>
 				</configuration>
 			</plugin>


[09/12] phoenix git commit: PHOENIX-2141 ComparisonExpression should return Boolean null if either operand is null

Posted by ja...@apache.org.
PHOENIX-2141 ComparisonExpression should return Boolean null if either operand is null


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

Branch: refs/heads/4.5-HBase-0.98
Commit: e78a6481e5910d3c57c62a621b8711d00e7a07f4
Parents: aa68b55
Author: maryannxue <we...@intel.com>
Authored: Mon Jul 27 14:50:16 2015 -0400
Committer: James Taylor <jt...@salesforce.com>
Committed: Tue Aug 11 18:36:52 2015 -0700

----------------------------------------------------------------------
 .../phoenix/compile/ExpressionCompiler.java     |  3 +
 .../apache/phoenix/compile/HavingCompiler.java  |  2 +-
 .../apache/phoenix/compile/WhereCompiler.java   |  2 +-
 .../apache/phoenix/compile/WhereOptimizer.java  |  2 +-
 .../phoenix/expression/AndExpression.java       |  3 +
 .../expression/ComparisonExpression.java        |  4 +-
 .../phoenix/expression/LiteralExpression.java   | 23 ++++++--
 .../phoenix/expression/NullValueTest.java       | 59 ++++++++++++++++++++
 8 files changed, 89 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/e78a6481/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java
index 1278494..1523dce 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java
@@ -251,6 +251,9 @@ public class ExpressionCompiler extends UnsupportedAllParseNodeVisitor<Expressio
             if (child.getDataType() != PBoolean.INSTANCE) {
                 throw TypeMismatchException.newException(PBoolean.INSTANCE, child.getDataType(), child.toString());
             }
+            if (LiteralExpression.isBooleanNull(child)) {
+                return child;
+            }
             if (LiteralExpression.isFalse(child)) {
                 iterator.remove();
             }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e78a6481/phoenix-core/src/main/java/org/apache/phoenix/compile/HavingCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/HavingCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/HavingCompiler.java
index 224a9b4..9ccd2f0 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/HavingCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/HavingCompiler.java
@@ -54,7 +54,7 @@ public class HavingCompiler {
         if (expression.getDataType() != PBoolean.INSTANCE) {
             throw TypeMismatchException.newException(PBoolean.INSTANCE, expression.getDataType(), expression.toString());
         }
-        if (LiteralExpression.isFalse(expression)) {
+        if (LiteralExpression.isBooleanFalseOrNull(expression)) {
             context.setScanRanges(ScanRanges.NOTHING);
             return null;
         } else if (LiteralExpression.isTrue(expression)) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e78a6481/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereCompiler.java
index 9631850..13963d7 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereCompiler.java
@@ -235,7 +235,7 @@ public class WhereCompiler {
     private static void setScanFilter(StatementContext context, FilterableStatement statement, Expression whereClause, boolean disambiguateWithFamily, boolean hashJoinOptimization) {
         Scan scan = context.getScan();
 
-        if (LiteralExpression.isFalse(whereClause)) {
+        if (LiteralExpression.isBooleanFalseOrNull(whereClause)) {
             context.setScanRanges(ScanRanges.NOTHING);
         } else if (whereClause != null && !LiteralExpression.isTrue(whereClause) && !hashJoinOptimization) {
             Filter filter = null;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e78a6481/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereOptimizer.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereOptimizer.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereOptimizer.java
index 6dfae7e..601eee1 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereOptimizer.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereOptimizer.java
@@ -120,7 +120,7 @@ public class WhereOptimizer {
             context.setScanRanges(ScanRanges.EVERYTHING);
             return whereClause;
         }
-        if (LiteralExpression.isFalse(whereClause)) {
+        if (LiteralExpression.isBooleanFalseOrNull(whereClause)) {
             context.setScanRanges(ScanRanges.NOTHING);
             return null;
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e78a6481/phoenix-core/src/main/java/org/apache/phoenix/expression/AndExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/AndExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/AndExpression.java
index 70e94ca..29b024d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/AndExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/AndExpression.java
@@ -44,6 +44,9 @@ public class AndExpression extends AndOrExpression {
             if (child.getDataType() != PBoolean.INSTANCE) {
                 throw TypeMismatchException.newException(PBoolean.INSTANCE, child.getDataType(), child.toString());
             }
+            if (LiteralExpression.isBooleanNull(child)) {
+                return child;
+            }
             if (LiteralExpression.isFalse(child)) {
                 return child;
             }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e78a6481/phoenix-core/src/main/java/org/apache/phoenix/expression/ComparisonExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/ComparisonExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/ComparisonExpression.java
index 1c8df20..074ac0a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/ComparisonExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/ComparisonExpression.java
@@ -142,7 +142,7 @@ public class ComparisonExpression extends BaseCompoundExpression {
         if (lhsExpr instanceof LiteralExpression) {
             lhsValue = ((LiteralExpression)lhsExpr).getValue();
             if (lhsValue == null) {
-                return LiteralExpression.newConstant(false, PBoolean.INSTANCE, lhsExpr.getDeterminism());
+                return LiteralExpression.newConstant(null, PBoolean.INSTANCE, lhsExpr.getDeterminism());
             }
         }
         Object rhsValue = null;
@@ -150,7 +150,7 @@ public class ComparisonExpression extends BaseCompoundExpression {
         if (rhsExpr instanceof LiteralExpression) {
             rhsValue = ((LiteralExpression)rhsExpr).getValue();
             if (rhsValue == null) {
-                return LiteralExpression.newConstant(false, PBoolean.INSTANCE, rhsExpr.getDeterminism());
+                return LiteralExpression.newConstant(null, PBoolean.INSTANCE, rhsExpr.getDeterminism());
             }
         }
         if (lhsValue != null && rhsValue != null) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e78a6481/phoenix-core/src/main/java/org/apache/phoenix/expression/LiteralExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/LiteralExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/LiteralExpression.java
index 05bd9b3..e911aae 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/LiteralExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/LiteralExpression.java
@@ -89,10 +89,10 @@ public class LiteralExpression extends BaseTerminalExpression {
     }
 
     public static boolean isFalse(Expression child) {
-    	if (child!=null) {
-    		return child == BOOLEAN_EXPRESSIONS[child.getDeterminism().ordinal()];
-    	}
-    	return false;
+        if (child!=null) {
+            return child == BOOLEAN_EXPRESSIONS[child.getDeterminism().ordinal()];
+        }
+        return false;
     }
     
     public static boolean isTrue(Expression child) {
@@ -101,6 +101,21 @@ public class LiteralExpression extends BaseTerminalExpression {
     	}
     	return false;
     }
+
+    public static boolean isBooleanNull(Expression child) {
+    	if (child!=null) {
+    		return child == TYPED_NULL_EXPRESSIONS[PBoolean.INSTANCE.ordinal()+PDataType.values().length*child.getDeterminism().ordinal()];
+    	}
+    	return false;
+    }
+
+    public static boolean isBooleanFalseOrNull(Expression child) {
+        if (child!=null) {
+            return child == BOOLEAN_EXPRESSIONS[child.getDeterminism().ordinal()]
+                    || child == TYPED_NULL_EXPRESSIONS[PBoolean.INSTANCE.ordinal()+PDataType.values().length*child.getDeterminism().ordinal()];
+        }
+        return false;
+    }
     
     public static LiteralExpression newConstant(Object value) {
         return newConstant(value, Determinism.ALWAYS);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e78a6481/phoenix-core/src/test/java/org/apache/phoenix/expression/NullValueTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/expression/NullValueTest.java b/phoenix-core/src/test/java/org/apache/phoenix/expression/NullValueTest.java
new file mode 100644
index 0000000..d5addf0
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/expression/NullValueTest.java
@@ -0,0 +1,59 @@
+package org.apache.phoenix.expression;
+
+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.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.util.Properties;
+
+import org.apache.phoenix.query.BaseConnectionlessQueryTest;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.junit.Test;
+
+public class NullValueTest extends BaseConnectionlessQueryTest {
+    
+    @Test
+    public void testComparisonExpressionWithNullOperands() throws Exception {
+        String[] query = {"SELECT 'a' >= ''", 
+                          "SELECT '' < 'a'", 
+                          "SELECT '' = ''"};
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            for (String q : query) {
+                ResultSet rs = conn.createStatement().executeQuery(q);
+                assertTrue(rs.next());
+                assertNull(rs.getObject(1));
+                assertEquals(false, rs.getBoolean(1));
+                assertFalse(rs.next());
+            }
+        } finally {
+            conn.close();
+        }       
+    }
+    
+    @Test
+    public void testAndOrExpressionWithNullOperands() throws Exception {
+        String[] query = {"SELECT 'a' >= '' or '' < 'a'", 
+                          "SELECT 'a' >= '' and '' < 'a'"};
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            for (String q : query) {
+                ResultSet rs = conn.createStatement().executeQuery(q);
+                assertTrue(rs.next());
+                assertNull(rs.getObject(1));
+                assertEquals(false, rs.getBoolean(1));
+                assertFalse(rs.next());
+            }
+        } finally {
+            conn.close();
+        }       
+    }
+
+}


[08/12] phoenix git commit: PHOENIX-1994 - TestIndexWriter#testFailureOnRunningUpdateAbortsPending() deadlocks on JDK 8

Posted by ja...@apache.org.
PHOENIX-1994 - TestIndexWriter#testFailureOnRunningUpdateAbortsPending() deadlocks on JDK 8


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

Branch: refs/heads/4.5-HBase-0.98
Commit: 233b48d6d0c9807172f4a617d75926ffa86f7e34
Parents: 42476da
Author: Cody Marcel <co...@gmail.com>
Authored: Mon Aug 10 12:08:07 2015 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Tue Aug 11 18:36:52 2015 -0700

----------------------------------------------------------------------
 .../java/org/apache/phoenix/hbase/index/write/TestIndexWriter.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/233b48d6/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestIndexWriter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestIndexWriter.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestIndexWriter.java
index 37a8cf4..8f576cf 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestIndexWriter.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestIndexWriter.java
@@ -145,7 +145,7 @@ public class TestIndexWriter {
     Abortable abort = new StubAbortable();
     Stoppable stop = Mockito.mock(Stoppable.class);
     // single thread factory so the older request gets queued
-    ExecutorService exec = Executors.newFixedThreadPool(1);
+    ExecutorService exec = Executors.newFixedThreadPool(3);
     Map<ImmutableBytesPtr, HTableInterface> tables = new HashMap<ImmutableBytesPtr, HTableInterface>();
     FakeTableFactory factory = new FakeTableFactory(tables);
 


[11/12] phoenix git commit: PHOENIX-2137 Range query on DECIMAL DESC sometimes incorrect

Posted by ja...@apache.org.
PHOENIX-2137 Range query on DECIMAL DESC sometimes incorrect

Conflicts:
	phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java


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

Branch: refs/heads/4.5-HBase-0.98
Commit: b6425149eeaf285cba18ea11057fde6db9fc0cdc
Parents: 233b48d
Author: James Taylor <jt...@salesforce.com>
Authored: Mon Aug 3 16:40:47 2015 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Tue Aug 11 18:36:52 2015 -0700

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/SortOrderIT.java | 110 ++++++++++++++++++-
 .../org/apache/phoenix/compile/ScanRanges.java  |   7 +-
 .../DescVarLengthFastByteComparisons.java       |  12 ++
 .../apache/phoenix/filter/SkipScanFilter.java   |  13 ++-
 .../java/org/apache/phoenix/query/KeyRange.java |  42 +++----
 .../apache/phoenix/schema/types/PDataType.java  |   3 +-
 .../java/org/apache/phoenix/util/ScanUtil.java  |  64 +++++++++--
 .../org/apache/phoenix/util/SchemaUtil.java     |   2 +-
 .../phoenix/compile/WhereOptimizerTest.java     |  20 ++++
 .../DescVarLengthFastByteComparisonsTest.java   |  45 ++++++++
 .../expression/SortOrderExpressionTest.java     |  13 ++-
 11 files changed, 288 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/b6425149/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderIT.java
index 0e8fb4f..9228ab5 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderIT.java
@@ -18,6 +18,7 @@
 package org.apache.phoenix.end2end;
 
 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;
 
@@ -28,12 +29,18 @@ import java.sql.Date;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
 import java.util.Calendar;
+import java.util.Collections;
 import java.util.GregorianCalendar;
 import java.util.List;
 import java.util.Properties;
 
 import org.apache.commons.lang.ArrayUtils;
+import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PDecimal;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.junit.Assert;
 import org.junit.Test;
@@ -357,6 +364,102 @@ public class SortOrderIT extends BaseHBaseManagedTimeIT {
                 null, null, new OrderBy("id", OrderBy.Direction.DESC));
     }
     
+    @Test
+    public void descVarLengthAscPKGT() throws Exception {
+        String ddl = "CREATE TABLE " + TABLE + " (k1 INTEGER NOT NULL, k2 VARCHAR, CONSTRAINT pk PRIMARY KEY (k1, k2))";
+        Object[][] insertedRows = new Object[][]{{0, null}, {1, "a"}, {2, "b"}, {3, "ba"}, {4, "baa"}, {5, "c"}, {6, "d"}};
+        Object[][] expectedRows = new Object[][]{{3}, {4}, {5}, {6}};
+        runQueryTest(ddl, upsert("k1", "k2"), select("k1"), insertedRows, expectedRows,
+                new WhereCondition("k2", ">", "'b'"), null, null);
+    }
+        
+    @Test
+    public void descVarLengthDescPKGT() throws Exception {
+        String ddl = "CREATE TABLE " + TABLE + " (k1 INTEGER NOT NULL, k2 VARCHAR, CONSTRAINT pk PRIMARY KEY (k1, k2 desc))";
+        Object[][] insertedRows = new Object[][]{{0, null}, {1, "a"}, {2, "b"}, {3, "ba"}, {4, "baa"}, {5, "c"}, {6, "d"}};
+        Object[][] expectedRows = new Object[][]{{3}, {4}, {5}, {6}};
+        runQueryTest(ddl, upsert("k1", "k2"), select("k1"), insertedRows, expectedRows,
+                new WhereCondition("k2", ">", "'b'"), null, null);
+    }
+        
+    @Test
+    public void descVarLengthDescPKLTE() throws Exception {
+        String ddl = "CREATE TABLE " + TABLE + " (k1 INTEGER NOT NULL, k2 VARCHAR, CONSTRAINT pk PRIMARY KEY (k1, k2 desc))";
+        Object[][] insertedRows = new Object[][]{{0, null}, {1, "a"}, {2, "b"}, {3, "ba"}, {4, "bb"}, {5, "bc"}, {6, "bba"}, {7, "c"}};
+        Object[][] expectedRows = new Object[][]{{1}, {2}, {3}, {4}};
+        runQueryTest(ddl, upsert("k1", "k2"), select("k1"), insertedRows, expectedRows,
+                new WhereCondition("k2", "<=", "'bb'"), null, null);
+    }
+        
+    @Test
+    public void descVarLengthAscPKLTE() throws Exception {
+        String ddl = "CREATE TABLE " + TABLE + " (k1 INTEGER NOT NULL, k2 VARCHAR, CONSTRAINT pk PRIMARY KEY (k1, k2))";
+        Object[][] insertedRows = new Object[][]{{0, null}, {1, "a"}, {2, "b"}, {3, "ba"}, {4, "bb"}, {5, "bc"}, {6, "bba"}, {7, "c"}};
+        Object[][] expectedRows = new Object[][]{{1}, {2}, {3}, {4}};
+        runQueryTest(ddl, upsert("k1", "k2"), select("k1"), insertedRows, expectedRows,
+                new WhereCondition("k2", "<=", "'bb'"), null, null);
+    }
+        
+   @Test
+    public void testNonPKCompare() throws Exception {
+        List<Integer> expectedResults = Lists.newArrayList(2,3,4);
+        Integer[] saltBuckets = new Integer[] {null,3};
+        PDataType[] dataTypes = new PDataType[] {PDecimal.INSTANCE};
+        for (Integer saltBucket : saltBuckets) {
+            for (PDataType dataType : dataTypes) {
+                for (SortOrder sortOrder : SortOrder.values()) {
+                    testCompareCompositeKey(saltBucket, dataType, sortOrder, "", expectedResults, "");
+                }
+            }
+        }
+    }
+
+    @Test
+    public void testSkipScanCompare() throws Exception {
+        List<Integer> expectedResults = Lists.newArrayList(2,4);
+        List<Integer> rExpectedResults = new ArrayList<>(expectedResults);
+        Collections.reverse(rExpectedResults);
+        Integer[] saltBuckets = new Integer[] {null,3};
+        PDataType[] dataTypes = new PDataType[] {PDecimal.INSTANCE};
+        for (Integer saltBucket : saltBuckets) {
+            for (PDataType dataType : dataTypes) {
+                for (SortOrder sortOrder : SortOrder.values()) {
+                    testCompareCompositeKey(saltBucket, dataType, sortOrder, "k1 in (2,4)", expectedResults, "");
+                    testCompareCompositeKey(saltBucket, dataType, sortOrder, "k1 in (2,4)", rExpectedResults, "ORDER BY k1 DESC");
+                }
+            }
+        }
+    }
+
+    private void testCompareCompositeKey(Integer saltBuckets, PDataType dataType, SortOrder sortOrder, String whereClause, List<Integer> expectedResults, String orderBy) throws SQLException {
+        String tableName = "t_" + saltBuckets + "_" + dataType + "_" + sortOrder;
+        String ddl = "create table if not exists " + tableName + " (k1 bigint not null, k2 " + dataType.getSqlTypeName() + (dataType.isFixedWidth() ? " not null" : "") + ", constraint pk primary key (k1,k2 " + sortOrder + "))" + (saltBuckets == null ? "" : (" SALT_BUCKETS= " + saltBuckets));
+        Connection conn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES));
+        conn.createStatement().execute(ddl);
+        if (!dataType.isFixedWidth()) {
+            conn.createStatement().execute("upsert into " + tableName + " values (0, null)");
+        }
+        conn.createStatement().execute("upsert into "  + tableName + " values (1, 0.99)");
+        conn.createStatement().execute("upsert into " + tableName + " values (2, 1.01)");
+        conn.createStatement().execute("upsert into "  + tableName + " values (3, 2.0)");
+        conn.createStatement().execute("upsert into " + tableName + " values (4, 1.001)");
+        conn.commit();
+
+        String query = "select k1 from " + tableName + " where " + (whereClause.length() > 0 ? (whereClause + " AND ") : "") + " k2>1.0 " + (orderBy.length() == 0 ? "" : orderBy);
+        try {
+            ResultSet rs = conn.createStatement().executeQuery(query);
+
+            for (int k : expectedResults) {
+                assertTrue (tableName, rs.next());
+                assertEquals(tableName, k,rs.getInt(1));
+            }
+
+            assertFalse(tableName, rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+
     private void runQueryTest(String ddl, String columnName, Object[][] rows, Object[][] expectedRows) throws Exception {
         runQueryTest(ddl, new String[]{columnName}, rows, expectedRows, null);
     }
@@ -546,8 +649,13 @@ public class SortOrderIT extends BaseHBaseManagedTimeIT {
                 return ">";
             } else if (operator.equals(">")) {
                 return "<";
+            } else if (operator.equals(">=")) {
+                return "<=";
+            } else if (operator.equals("<=")) {
+                return ">=";
+            } else {
+                return operator;
             }
-            return operator;
         }
     }
     

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b6425149/phoenix-core/src/main/java/org/apache/phoenix/compile/ScanRanges.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/ScanRanges.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/ScanRanges.java
index 80cfbfe..298cd4e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/ScanRanges.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/ScanRanges.java
@@ -37,6 +37,7 @@ import org.apache.phoenix.schema.RowKeySchema;
 import org.apache.phoenix.schema.SaltingUtil;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.ScanUtil;
+import org.apache.phoenix.util.ScanUtil.BytesComparator;
 import org.apache.phoenix.util.SchemaUtil;
 
 import com.google.common.collect.ImmutableList;
@@ -75,10 +76,12 @@ public class ScanRanges {
                         stripPrefix(minMaxRange.getUpperRange(),offset), 
                         minMaxRange.upperUnbound());
             }
+            // We have full keys here, so use field from our varbinary schema
+            BytesComparator comparator = ScanUtil.getComparator(SchemaUtil.VAR_BINARY_SCHEMA.getField(0));
             for (byte[] key : keys) {
                 // Filter now based on unsalted minMaxRange and ignore the point key salt byte
-                if ( unsaltedMinMaxRange.compareLowerToUpperBound(key, offset, key.length-offset, true) <= 0 &&
-                     unsaltedMinMaxRange.compareUpperToLowerBound(key, offset, key.length-offset, true) >= 0) {
+                if ( unsaltedMinMaxRange.compareLowerToUpperBound(key, offset, key.length-offset, true, comparator) <= 0 &&
+                     unsaltedMinMaxRange.compareUpperToLowerBound(key, offset, key.length-offset, true, comparator) >= 0) {
                     keyRanges.add(KeyRange.getKeyRange(key));
                 }
             }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b6425149/phoenix-core/src/main/java/org/apache/phoenix/execute/DescVarLengthFastByteComparisons.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/DescVarLengthFastByteComparisons.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/DescVarLengthFastByteComparisons.java
index 40960e0..67d23fc 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/DescVarLengthFastByteComparisons.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/DescVarLengthFastByteComparisons.java
@@ -90,6 +90,12 @@ public class DescVarLengthFastByteComparisons {
             public int compareTo(byte[] buffer1, int offset1, int length1, byte[] buffer2, int offset2, int length2) {
                 // Short circuit equal case
                 if (buffer1 == buffer2 && offset1 == offset2 && length1 == length2) { return 0; }
+                if (length1 == 0 && length2 != 0) { // nulls sort first, even for descending
+                    return -1; 
+                } 
+                if (length2 == 0 && length1 != 0) { // nulls sort first, even for descending
+                    return 1; 
+                }
                 // Bring WritableComparator code local
                 int end1 = offset1 + length1;
                 int end2 = offset2 + length2;
@@ -166,6 +172,12 @@ public class DescVarLengthFastByteComparisons {
             public int compareTo(byte[] buffer1, int offset1, int length1, byte[] buffer2, int offset2, int length2) {
                 // Short circuit equal case
                 if (buffer1 == buffer2 && offset1 == offset2 && length1 == length2) { return 0; }
+                if (length1 == 0 && length2 != 0) { // nulls sort first, even for descending
+                    return -1; 
+                }
+                if (length2 == 0 && length1 != 0) { // nulls sort first, even for descending
+                    return 1; 
+                }
                 int minLength = Math.min(length1, length2);
                 int minWords = minLength / Longs.BYTES;
                 int offset1Adj = offset1 + BYTE_ARRAY_BASE_OFFSET;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b6425149/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 4dc888d..ff58a18 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
@@ -40,6 +40,7 @@ import org.apache.phoenix.query.KeyRange.Bound;
 import org.apache.phoenix.schema.RowKeySchema;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.ScanUtil;
+import org.apache.phoenix.util.ScanUtil.BytesComparator;
 import org.apache.phoenix.util.SchemaUtil;
 
 import com.google.common.base.Objects;
@@ -202,7 +203,7 @@ public class SkipScanFilter extends FilterBase implements Writable {
         if (!lowerUnbound) {
             // Find the position of the first slot of the lower range
             schema.next(ptr, 0, schema.iterator(lowerInclusiveKey,ptr), slotSpan[0]);
-            startPos = ScanUtil.searchClosestKeyRangeWithUpperHigherThanPtr(slots.get(0), ptr, 0);
+            startPos = ScanUtil.searchClosestKeyRangeWithUpperHigherThanPtr(slots.get(0), ptr, 0, schema.getField(0));
             // Lower range is past last upper range of first slot, so cannot possibly be in range
             if (startPos >= slots.get(0).size()) {
                 return false;
@@ -213,7 +214,7 @@ public class SkipScanFilter extends FilterBase implements Writable {
         if (!upperUnbound) {
             // Find the position of the first slot of the upper range
             schema.next(ptr, 0, schema.iterator(upperExclusiveKey,ptr), slotSpan[0]);
-            endPos = ScanUtil.searchClosestKeyRangeWithUpperHigherThanPtr(slots.get(0), ptr, startPos);
+            endPos = ScanUtil.searchClosestKeyRangeWithUpperHigherThanPtr(slots.get(0), ptr, startPos, schema.getField(0));
             // Upper range lower than first lower range of first slot, so cannot possibly be in range
 //            if (endPos == 0 && Bytes.compareTo(upperExclusiveKey, slots.get(0).get(0).getLowerRange()) <= 0) {
 //                return false;
@@ -222,7 +223,7 @@ public class SkipScanFilter extends FilterBase implements Writable {
             if (endPos >= slots.get(0).size()) {
                 upperUnbound = true;
                 endPos = slots.get(0).size()-1;
-            } else if (slots.get(0).get(endPos).compareLowerToUpperBound(upperExclusiveKey) >= 0) {
+            } else if (slots.get(0).get(endPos).compareLowerToUpperBound(upperExclusiveKey, ScanUtil.getComparator(schema.getField(0))) >= 0) {
                 // We know that the endPos range is higher than the previous range, but we need
                 // to test if it ends before the next range starts.
                 endPos--;
@@ -389,8 +390,10 @@ public class SkipScanFilter extends FilterBase implements Writable {
         int maxOffset = schema.iterator(currentKey, minOffset, length, ptr);
         schema.next(ptr, ScanUtil.getRowKeyPosition(slotSpan, i), maxOffset, slotSpan[i]);
         while (true) {
+            // Comparator depends on field in schema
+            BytesComparator comparator = ScanUtil.getComparator(schema.getField(ScanUtil.getRowKeyPosition(slotSpan, i)));
             // Increment to the next range while the upper bound of our current slot is less than our current key
-            while (position[i] < slots.get(i).size() && slots.get(i).get(position[i]).compareUpperToLowerBound(ptr) < 0) {
+            while (position[i] < slots.get(i).size() && slots.get(i).get(position[i]).compareUpperToLowerBound(ptr, comparator) < 0) {
                 position[i]++;
             }
             Arrays.fill(position, i+1, position.length, 0);
@@ -440,7 +443,7 @@ public class SkipScanFilter extends FilterBase implements Writable {
                     ByteUtil.nextKey(startKey, currentLength);
                 }
                 i = j;
-            } else if (slots.get(i).get(position[i]).compareLowerToUpperBound(ptr) > 0) {
+            } 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);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b6425149/phoenix-core/src/main/java/org/apache/phoenix/query/KeyRange.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/KeyRange.java b/phoenix-core/src/main/java/org/apache/phoenix/query/KeyRange.java
index 0612046..366a123 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/KeyRange.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/KeyRange.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.ScanUtil.BytesComparator;
 
 import com.google.common.base.Function;
 import com.google.common.collect.ComparisonChain;
@@ -183,28 +184,28 @@ public class KeyRange implements Writable {
         return isSingleKey;
     }
     
-    public int compareLowerToUpperBound(ImmutableBytesWritable ptr, boolean isInclusive) {
-        return compareLowerToUpperBound(ptr.get(), ptr.getOffset(), ptr.getLength(), isInclusive);
+    public int compareLowerToUpperBound(ImmutableBytesWritable ptr, boolean isInclusive, BytesComparator comparator) {
+        return compareLowerToUpperBound(ptr.get(), ptr.getOffset(), ptr.getLength(), isInclusive, comparator);
     }
     
-    public int compareLowerToUpperBound(ImmutableBytesWritable ptr) {
-        return compareLowerToUpperBound(ptr, true);
+    public int compareLowerToUpperBound(ImmutableBytesWritable ptr, BytesComparator comparator) {
+        return compareLowerToUpperBound(ptr, true, comparator);
     }
     
-    public int compareUpperToLowerBound(ImmutableBytesWritable ptr, boolean isInclusive) {
-        return compareUpperToLowerBound(ptr.get(), ptr.getOffset(), ptr.getLength(), isInclusive);
+    public int compareUpperToLowerBound(ImmutableBytesWritable ptr, boolean isInclusive, BytesComparator comparator) {
+        return compareUpperToLowerBound(ptr.get(), ptr.getOffset(), ptr.getLength(), isInclusive, comparator);
     }
     
-    public int compareUpperToLowerBound(ImmutableBytesWritable ptr) {
-        return compareUpperToLowerBound(ptr, true);
+    public int compareUpperToLowerBound(ImmutableBytesWritable ptr, BytesComparator comparator) {
+        return compareUpperToLowerBound(ptr, true, comparator);
     }
     
-    public int compareLowerToUpperBound( byte[] b, int o, int l) {
-        return compareLowerToUpperBound(b,o,l,true);
+    public int compareLowerToUpperBound( byte[] b, int o, int l, BytesComparator comparator) {
+        return compareLowerToUpperBound(b,o,l,true, comparator);
     }
 
-    public int compareLowerToUpperBound( byte[] b) {
-        return compareLowerToUpperBound(b,0,b.length);
+    public int compareLowerToUpperBound( byte[] b, BytesComparator comparator) {
+        return compareLowerToUpperBound(b,0,b.length, comparator);
     }
 
     /**
@@ -213,15 +214,16 @@ public class KeyRange implements Writable {
      * @param o upper bound offset
      * @param l upper bound length
      * @param isInclusive upper bound inclusive
+     * @param comparator comparator used to do compare the byte array using offset and length
      * @return -1 if the lower bound is less than the upper bound,
      *          1 if the lower bound is greater than the upper bound,
      *          and 0 if they are equal.
      */
-    public int compareLowerToUpperBound( byte[] b, int o, int l, boolean isInclusive) {
+    public int compareLowerToUpperBound( byte[] b, int o, int l, boolean isInclusive, BytesComparator comparator) {
         if (lowerUnbound() || b == KeyRange.UNBOUND) {
             return -1;
         }
-        int cmp = Bytes.compareTo(lowerRange, 0, lowerRange.length, b, o, l);
+        int cmp = comparator.compare(lowerRange, 0, lowerRange.length, b, o, l);
         if (cmp > 0) {
             return 1;
         }
@@ -234,19 +236,19 @@ public class KeyRange implements Writable {
         return 1;
     }
     
-    public int compareUpperToLowerBound(byte[] b) {
-        return compareUpperToLowerBound(b,0,b.length);
+    public int compareUpperToLowerBound(byte[] b, BytesComparator comparator) {
+        return compareUpperToLowerBound(b,0,b.length, comparator);
     }
     
-    public int compareUpperToLowerBound(byte[] b, int o, int l) {
-        return compareUpperToLowerBound(b,o,l, true);
+    public int compareUpperToLowerBound(byte[] b, int o, int l, BytesComparator comparator) {
+        return compareUpperToLowerBound(b,o,l, true, comparator);
     }
     
-    public int compareUpperToLowerBound(byte[] b, int o, int l, boolean isInclusive) {
+    public int compareUpperToLowerBound(byte[] b, int o, int l, boolean isInclusive, BytesComparator comparator) {
         if (upperUnbound() || b == KeyRange.UNBOUND) {
             return 1;
         }
-        int cmp = Bytes.compareTo(upperRange, 0, upperRange.length, b, o, l);
+        int cmp = comparator.compare(upperRange, 0, upperRange.length, b, o, l);
         if (cmp > 0) {
             return 1;
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b6425149/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDataType.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDataType.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDataType.java
index 43bab0e..d79de60 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDataType.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDataType.java
@@ -36,6 +36,7 @@ import org.apache.phoenix.schema.ConstraintViolationException;
 import org.apache.phoenix.schema.IllegalDataException;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.ScanUtil;
 
 import com.google.common.base.Preconditions;
 import com.google.common.math.LongMath;
@@ -762,7 +763,7 @@ public abstract class PDataType<T> implements DataType<T>, Comparable<PDataType<
             }
             return (length1 - length2);
         }
-        return Bytes.compareTo(ba1, offset1, length1, ba2, offset2, length2) * (so1 == SortOrder.DESC ? -1 : 1);
+        return (so1 == SortOrder.DESC ? -1 : 1) * ScanUtil.getComparator(length1 == length2, so1).compare(ba1, offset1, length1, ba2, offset2, length2);
     }
 
     public final int compareTo(ImmutableBytesWritable ptr1, SortOrder ptr1SortOrder, ImmutableBytesWritable ptr2,

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b6425149/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java
index 9d104ca..ae073e2 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java
@@ -40,11 +40,13 @@ import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.io.TimeRange;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.WritableComparator;
 import org.apache.phoenix.compile.OrderByCompiler.OrderBy;
 import org.apache.phoenix.compile.ScanRanges;
 import org.apache.phoenix.compile.StatementContext;
 import org.apache.phoenix.coprocessor.BaseScannerRegionObserver;
 import org.apache.phoenix.coprocessor.MetaDataProtocol;
+import org.apache.phoenix.execute.DescVarLengthFastByteComparisons;
 import org.apache.phoenix.filter.BooleanExpressionFilter;
 import org.apache.phoenix.filter.SkipScanFilter;
 import org.apache.phoenix.query.KeyRange;
@@ -55,6 +57,7 @@ import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PNameFactory;
 import org.apache.phoenix.schema.RowKeySchema;
+import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.ValueSchema.Field;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PVarbinary;
@@ -283,7 +286,15 @@ public class ScanUtil {
         for (int i = 0; i < position.length; i++) {
             position[i] = bound == Bound.LOWER ? 0 : slots.get(i).size()-1;
             KeyRange range = slots.get(i).get(position[i]);
-            maxLength += range.getRange(bound).length + (schema.getField(i + slotSpan[i]).getDataType().isFixedWidth() ? 0 : 1);
+            Field field = schema.getField(i + slotSpan[i]);
+            int keyLength = range.getRange(bound).length;
+            if (!field.getDataType().isFixedWidth()) {
+                keyLength++;
+                if (range.isUnbound(bound) && !range.isInclusive(bound) && field.getSortOrder() == SortOrder.DESC) {
+                    keyLength++;
+                }
+            }
+            maxLength += keyLength;
         }
         byte[] key = new byte[maxLength];
         int length = setKey(schema, slots, slotSpan, position, bound, key, 0, 0, position.length);
@@ -371,8 +382,8 @@ public class ScanUtil {
             // key slots would cause the flag to become true.
             lastInclusiveUpperSingleKey = range.isSingleKey() && inclusiveUpper;
             anyInclusiveUpperRangeKey |= !range.isSingleKey() && inclusiveUpper;
-            // A match for IS NULL or IS NOT NULL should not have a DESC_SEPARATOR_BYTE as nulls sort first
-            byte sepByte = SchemaUtil.getSeparatorByte(schema.rowKeyOrderOptimizable(), bytes.length == 0 || range == KeyRange.IS_NULL_RANGE || range == KeyRange.IS_NOT_NULL_RANGE, field);
+            // A null or empty byte array is always represented as a zero byte
+            byte sepByte = SchemaUtil.getSeparatorByte(schema.rowKeyOrderOptimizable(), bytes.length == 0, field);
             
             if (!isFixedWidth && ( fieldIndex < schema.getMaxFields() || inclusiveUpper || exclusiveLower || sepByte == QueryConstants.DESC_SEPARATOR_BYTE)) {
                 key[offset++] = sepByte;
@@ -383,7 +394,7 @@ public class ScanUtil {
             // If we are setting the lower bound with an exclusive range key, we need to bump the
             // slot up for each key part. For an upper bound, we bump up an inclusive key, but
             // only after the last key part.
-            if (!range.isSingleKey() && exclusiveLower) {
+            if (exclusiveLower) {
                 if (!ByteUtil.nextKey(key, offset)) {
                     // Special case for not being able to increment.
                     // In this case we return a negative byteOffset to
@@ -392,6 +403,14 @@ public class ScanUtil {
                     // have an end key specified.
                     return -byteOffset;
                 }
+                // We're filtering on values being non null here, but we still need the 0xFF
+                // terminator, since DESC keys ignore the last byte as it's expected to be 
+                // the terminator. Without this, we'd ignore the separator byte that was
+                // just added and incremented.
+                if (!isFixedWidth && bytes.length == 0 
+                    && SchemaUtil.getSeparatorByte(schema.rowKeyOrderOptimizable(), false, field) == QueryConstants.DESC_SEPARATOR_BYTE) {
+                    key[offset++] = QueryConstants.DESC_SEPARATOR_BYTE;
+                }
             }
         }
         if (lastInclusiveUpperSingleKey || anyInclusiveUpperRangeKey) {
@@ -409,7 +428,8 @@ public class ScanUtil {
         // byte.
         if (bound == Bound.LOWER) {
             while (--i >= schemaStartIndex && offset > byteOffset && 
-                    !schema.getField(--fieldIndex).getDataType().isFixedWidth() && 
+                    !(field=schema.getField(--fieldIndex)).getDataType().isFixedWidth() && 
+                    field.getSortOrder() == SortOrder.ASC &&
                     key[offset-1] == QueryConstants.SEPARATOR_BYTE) {
                 offset--;
                 fieldIndex -= slotSpan[i];
@@ -417,19 +437,47 @@ public class ScanUtil {
         }
         return offset - byteOffset;
     }
+    
+    public static interface BytesComparator {
+        public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2);
+    };
 
+    private static final BytesComparator DESC_VAR_WIDTH_COMPARATOR = new BytesComparator() {
+
+        @Override
+        public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+            return DescVarLengthFastByteComparisons.compareTo(b1, s1, l1, b2, s2, l2);
+        }
+        
+    };
+    
+    private static final BytesComparator ASC_FIXED_WIDTH_COMPARATOR = new BytesComparator() {
+
+        @Override
+        public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+            return WritableComparator.compareBytes(b1, s1, l1, b2, s2, l2);
+        }
+        
+    };
+    public static BytesComparator getComparator(boolean isFixedWidth, SortOrder sortOrder) {
+        return isFixedWidth || sortOrder == SortOrder.ASC ? ASC_FIXED_WIDTH_COMPARATOR : DESC_VAR_WIDTH_COMPARATOR;
+    }
+    public static BytesComparator getComparator(Field field) {
+        return getComparator(field.getDataType().isFixedWidth(),field.getSortOrder());
+    }
     /**
      * Perform a binary lookup on the list of KeyRange for the tightest slot such that the slotBound
      * of the current slot is higher or equal than the slotBound of our range. 
      * @return  the index of the slot whose slot bound equals or are the tightest one that is 
      *          smaller than rangeBound of range, or slots.length if no bound can be found.
      */
-    public static int searchClosestKeyRangeWithUpperHigherThanPtr(List<KeyRange> slots, ImmutableBytesWritable ptr, int lower) {
+    public static int searchClosestKeyRangeWithUpperHigherThanPtr(List<KeyRange> slots, ImmutableBytesWritable ptr, int lower, Field field) {
         int upper = slots.size() - 1;
         int mid;
+        BytesComparator comparator = ScanUtil.getComparator(field.getDataType().isFixedWidth(), field.getSortOrder());
         while (lower <= upper) {
             mid = (lower + upper) / 2;
-            int cmp = slots.get(mid).compareUpperToLowerBound(ptr, true);
+            int cmp = slots.get(mid).compareUpperToLowerBound(ptr, true, comparator);
             if (cmp < 0) {
                 lower = mid + 1;
             } else if (cmp > 0) {
@@ -439,7 +487,7 @@ public class ScanUtil {
             }
         }
         mid = (lower + upper) / 2;
-        if (mid == 0 && slots.get(mid).compareUpperToLowerBound(ptr, true) > 0) {
+        if (mid == 0 && slots.get(mid).compareUpperToLowerBound(ptr, true, comparator) > 0) {
             return mid;
         } else {
             return ++mid;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b6425149/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
index d01bf39..5414d4f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
@@ -625,7 +625,7 @@ public class SchemaUtil {
     }
 
     public static int getMaxKeyLength(RowKeySchema schema, List<List<KeyRange>> slots) {
-        int maxKeyLength = getTerminatorCount(schema);
+        int maxKeyLength = getTerminatorCount(schema) * 2;
         for (List<KeyRange> slot : slots) {
             int maxSlotLength = 0;
             for (KeyRange range : slot) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b6425149/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java
index c1787ca..2e0a0c1 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java
@@ -31,6 +31,7 @@ import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+import java.math.BigDecimal;
 import java.sql.Connection;
 import java.sql.Date;
 import java.sql.DriverManager;
@@ -55,9 +56,12 @@ import org.apache.phoenix.query.BaseConnectionlessQueryTest;
 import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.ColumnNotFoundException;
+import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.types.PChar;
 import org.apache.phoenix.schema.types.PDate;
+import org.apache.phoenix.schema.types.PDecimal;
 import org.apache.phoenix.schema.types.PInteger;
+import org.apache.phoenix.schema.types.PLong;
 import org.apache.phoenix.schema.types.PUnsignedLong;
 import org.apache.phoenix.schema.types.PVarchar;
 import org.apache.phoenix.util.ByteUtil;
@@ -106,6 +110,22 @@ public class WhereOptimizerTest extends BaseConnectionlessQueryTest {
     }
 
     @Test
+    public void testDescDecimalRange() throws SQLException {
+        String ddl = "create table t (k1 bigint not null, k2 decimal, constraint pk primary key (k1,k2 desc))";
+        Connection conn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES));
+        conn.createStatement().execute(ddl);
+        String query = "select * from t where k1 in (1,2) and k2>1.0";
+        Scan scan = compileStatement(query).getScan();
+
+        byte[] startRow = ByteUtil.concat(PLong.INSTANCE.toBytes(1), ByteUtil.nextKey(QueryConstants.SEPARATOR_BYTE_ARRAY), QueryConstants.DESC_SEPARATOR_BYTE_ARRAY);
+        byte[] upperValue = PDecimal.INSTANCE.toBytes(BigDecimal.valueOf(1.0));
+        byte[] stopRow = ByteUtil.concat(PLong.INSTANCE.toBytes(2), SortOrder.invert(upperValue,0,upperValue.length), QueryConstants.DESC_SEPARATOR_BYTE_ARRAY);
+        assertTrue(scan.getFilter() instanceof SkipScanFilter);
+        assertArrayEquals(startRow, scan.getStartRow());
+        assertArrayEquals(stopRow, scan.getStopRow());
+    }
+
+    @Test
     public void testSingleCharPaddedKeyExpression() throws SQLException {
         String tenantId = "1";
         String query = "select * from atable where organization_id='" + tenantId + "'";

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b6425149/phoenix-core/src/test/java/org/apache/phoenix/execute/DescVarLengthFastByteComparisonsTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/execute/DescVarLengthFastByteComparisonsTest.java b/phoenix-core/src/test/java/org/apache/phoenix/execute/DescVarLengthFastByteComparisonsTest.java
new file mode 100644
index 0000000..106471b
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/execute/DescVarLengthFastByteComparisonsTest.java
@@ -0,0 +1,45 @@
+/*
+ * 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.execute;
+
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.util.ByteUtil;
+import org.junit.Test;
+
+public class DescVarLengthFastByteComparisonsTest {
+    
+    @Test
+    public void testNullIsSmallest() {
+        byte[] b1 = ByteUtil.EMPTY_BYTE_ARRAY;
+        byte[] b2 = Bytes.toBytes("a");
+        int cmp = DescVarLengthFastByteComparisons.compareTo(b1, 0, b1.length, b2, 0, b2.length);
+        assertTrue(cmp < 0);
+        cmp = DescVarLengthFastByteComparisons.compareTo(b2, 0, b2.length, b1, 0, b1.length);
+        assertTrue(cmp > 0);
+    }
+    
+    @Test
+    public void testShorterSubstringIsBigger() {
+        byte[] b1 = Bytes.toBytes("ab");
+        byte[] b2 = Bytes.toBytes("a");
+        int cmp = DescVarLengthFastByteComparisons.compareTo(b1, 0, b1.length, b2, 0, b2.length);
+        assertTrue(cmp < 0);
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b6425149/phoenix-core/src/test/java/org/apache/phoenix/expression/SortOrderExpressionTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/expression/SortOrderExpressionTest.java b/phoenix-core/src/test/java/org/apache/phoenix/expression/SortOrderExpressionTest.java
index b9ee0eb..e2ab684 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/expression/SortOrderExpressionTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/expression/SortOrderExpressionTest.java
@@ -292,17 +292,20 @@ public class SortOrderExpressionTest {
     }
     
     private void runCompareTest(CompareOp op, boolean expectedResult, Object lhsValue, PDataType lhsDataType, Object rhsValue, PDataType rhsDataType) throws Exception {
-        List<Expression> args = Lists.newArrayList(getLiteral(lhsValue, lhsDataType), getLiteral(rhsValue, rhsDataType));
-        evaluateAndAssertResult(new ComparisonExpression(args, op), expectedResult, "lhsDataType: " + lhsDataType + " rhsDataType: " + rhsDataType);
+        List<Expression> args;
+        ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+
+        args = Lists.newArrayList(getLiteral(lhsValue, lhsDataType), getLiteral(rhsValue, rhsDataType));
+        evaluateAndAssertResult(ComparisonExpression.create(op, args, ptr, true), expectedResult, "lhsDataType: " + lhsDataType + " rhsDataType: " + rhsDataType);
         
         args = Lists.newArrayList(getInvertedLiteral(lhsValue, lhsDataType), getLiteral(rhsValue, rhsDataType));
-        evaluateAndAssertResult(new ComparisonExpression(args, op), expectedResult, "lhs (inverted) dataType: " + lhsDataType + " rhsDataType: " + rhsDataType);
+        evaluateAndAssertResult(ComparisonExpression.create(op, args, ptr, true), expectedResult, "lhs (inverted) dataType: " + lhsDataType + " rhsDataType: " + rhsDataType);
         
         args = Lists.newArrayList(getLiteral(lhsValue, lhsDataType), getInvertedLiteral(rhsValue, rhsDataType));
-        evaluateAndAssertResult(new ComparisonExpression(args, op), expectedResult, "lhsDataType: " + lhsDataType + " rhs (inverted) dataType: " + rhsDataType);
+        evaluateAndAssertResult(ComparisonExpression.create(op, args, ptr, true), expectedResult, "lhsDataType: " + lhsDataType + " rhs (inverted) dataType: " + rhsDataType);
         
         args = Lists.newArrayList(getInvertedLiteral(lhsValue, lhsDataType), getInvertedLiteral(rhsValue, rhsDataType));
-        evaluateAndAssertResult(new ComparisonExpression(args, op), expectedResult, "lhs (inverted) dataType: " + lhsDataType + " rhs (inverted) dataType: " + rhsDataType);                
+        evaluateAndAssertResult(ComparisonExpression.create(op, args, ptr, true), expectedResult, "lhs (inverted) dataType: " + lhsDataType + " rhs (inverted) dataType: " + rhsDataType);                
     }
     
     private void evaluateAndAssertResult(Expression expression, Object expectedResult) {


[05/12] phoenix git commit: PHOENIX-2132 Pherf - Fix drop all command and execution from Eclipse/IDE

Posted by ja...@apache.org.
PHOENIX-2132 Pherf - Fix drop all command and execution from Eclipse/IDE


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

Branch: refs/heads/4.5-HBase-0.98
Commit: fd7523fa39a98739a17112cbd03ff99d3a15a77a
Parents: 43cd3ae
Author: Mujtaba <mu...@apache.org>
Authored: Fri Jul 24 11:18:52 2015 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Tue Aug 11 18:36:52 2015 -0700

----------------------------------------------------------------------
 phoenix-pherf/pom.xml                           | 25 ++++++---
 .../apache/phoenix/pherf/util/PhoenixUtil.java  | 55 +++++++++++++++-----
 2 files changed, 60 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/fd7523fa/phoenix-pherf/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-pherf/pom.xml b/phoenix-pherf/pom.xml
index a8c2979..6824b1f 100644
--- a/phoenix-pherf/pom.xml
+++ b/phoenix-pherf/pom.xml
@@ -69,7 +69,6 @@
 		<dependency>
 			<groupId>junit</groupId>
 			<artifactId>junit</artifactId>
-			<version>4.11</version>
 			<scope>test</scope>
 		</dependency>
 		<dependency>
@@ -91,14 +90,12 @@
 		<dependency>
 			<groupId>org.apache.hbase</groupId>
 			<artifactId>hbase-testing-util</artifactId>
-			<version>${hbase.version}</version>
 			<scope>test</scope>
 			<optional>true</optional>
 		</dependency>
 		<dependency>
 			<groupId>org.apache.hbase</groupId>
 			<artifactId>hbase-it</artifactId>
-			<version>${hbase.version}</version>
 			<type>test-jar</type>
 			<scope>test</scope>
 		</dependency>
@@ -109,6 +106,12 @@
 			<resource>
 				<directory>src/main/resources</directory>
 			</resource>
+			<resource>
+				<directory>config</directory>
+				<includes>
+					<include>**/*.properties</include>
+				</includes>
+			</resource>
 		</resources>
 		<testResources>
 			<testResource>
@@ -129,13 +132,11 @@
 			</plugin>
 			<plugin>
 				<artifactId>maven-dependency-plugin</artifactId>
-				<version>${maven-dependency-plugin.version}</version>
 			</plugin>
 
 			<plugin>
 				<groupId>org.apache.maven.plugins</groupId>
 				<artifactId>maven-compiler-plugin</artifactId>
-				<version>3.1</version>
 				<configuration>
 					<source>1.7</source>
 					<target>1.7</target>
@@ -177,17 +178,27 @@
 						</goals>
 						<configuration>
 							<descriptors>
-							    <!-- Produces minimal Pherf jar -->
+								<!-- Produces minimal Pherf jar -->
 								<descriptor>src/main/assembly/minimal.xml</descriptor>
 								<!-- Produces standalone zip that bundles all required dependencies -->
 								<descriptor>src/main/assembly/standalone.xml</descriptor>
-								<!-- Produces cluster zip with minimal Pherf jar. Setting HBase classpath in env.sh is required for this configuration -->								
+								<!-- Produces cluster zip with minimal Pherf jar. Setting HBase classpath 
+									in env.sh is required for this configuration -->
 								<descriptor>src/main/assembly/cluster.xml</descriptor>
 							</descriptors>
 						</configuration>
 					</execution>
 				</executions>
 			</plugin>
+			<plugin>
+				<groupId>org.apache.rat</groupId>
+				<artifactId>apache-rat-plugin</artifactId>
+				<configuration>
+					<excludes>
+						<exclude>RESULTS/**</exclude>
+					</excludes>
+				</configuration>
+			</plugin>
 		</plugins>
 	</build>
 </project>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fd7523fa/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/util/PhoenixUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/util/PhoenixUtil.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/util/PhoenixUtil.java
index 0156149..5f7d637 100644
--- a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/util/PhoenixUtil.java
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/util/PhoenixUtil.java
@@ -22,19 +22,17 @@ import org.apache.phoenix.pherf.PherfConstants;
 import org.apache.phoenix.pherf.configuration.Column;
 import org.apache.phoenix.pherf.configuration.DataTypeMapping;
 
-import java.io.File;
-import java.io.IOException;
-import java.io.OutputStream;
 import java.sql.*;
 import java.util.*;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.phoenix.pherf.configuration.Query;
 import org.apache.phoenix.pherf.configuration.QuerySet;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SCHEM;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_NAME;
+
 // TODO This class needs to be cleanup up a bit. I just wanted to get an initial placeholder in.
 public class PhoenixUtil {
 	private static final Logger logger = LoggerFactory.getLogger(PhoenixUtil.class);
@@ -95,6 +93,26 @@ public class PhoenixUtil {
         }
         return result;
     }
+
+    /**
+     * Execute statement
+     * @param sql
+     * @param connection
+     * @return
+     * @throws SQLException
+     */
+    public boolean executeStatementThrowException(String sql, Connection connection) throws SQLException {
+    	boolean result = false;
+    	PreparedStatement preparedStatement = null;
+    	try {
+            preparedStatement = connection.prepareStatement(sql);
+            result = preparedStatement.execute();
+            connection.commit();
+        } finally {
+            preparedStatement.close();
+        }
+        return result;
+    }
     
     public boolean executeStatement(String sql, Connection connection) {
     	boolean result = false;
@@ -139,14 +157,25 @@ public class PhoenixUtil {
     	Connection conn = getConnection();
     	try {
         	ResultSet resultSet = getTableMetaData(PherfConstants.PHERF_SCHEMA_NAME, null, conn);
-	    	while (resultSet.next()) {
-	    		String tableName = resultSet.getString("TABLE_SCHEMA") == null ? resultSet.getString("TABLE_NAME") :
-	    						   resultSet.getString("TABLE_SCHEMA") + "." + resultSet.getString("TABLE_NAME");
-	    		if (tableName.matches(regexMatch)) {
-		    		logger.info("\nDropping " + tableName);
-		    		executeStatement("DROP TABLE " + tableName + " CASCADE", conn);
-	    		}
-	    	}
+			while (resultSet.next()) {
+				String tableName = resultSet.getString(TABLE_SCHEM) == null ? resultSet
+						.getString(TABLE_NAME) : resultSet
+						.getString(TABLE_SCHEM)
+						+ "."
+						+ resultSet.getString(TABLE_NAME);
+				if (tableName.matches(regexMatch)) {
+					logger.info("\nDropping " + tableName);
+					try {
+						executeStatementThrowException("DROP TABLE "
+								+ tableName + " CASCADE", conn);
+					} catch (org.apache.phoenix.schema.TableNotFoundException tnf) {
+						logger.error("Table might be already be deleted via cascade. Schema: "
+								+ tnf.getSchemaName()
+								+ " Table: "
+								+ tnf.getTableName());
+					}
+				}
+			}
     	} finally {
     		conn.close();
     	}


[07/12] phoenix git commit: PHOENIX-2138 Non equality comparisons don't work for ARRAY type columns that are DESC in row key (Dumindu Buddhika)

Posted by ja...@apache.org.
PHOENIX-2138 Non equality comparisons don't work for ARRAY type columns
that are DESC in row key (Dumindu Buddhika)


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

Branch: refs/heads/4.5-HBase-0.98
Commit: aa68b5568483f801bc0fc71ad94bfb3d452b4625
Parents: b1ad90f
Author: ramkrishna <ra...@gmail.com>
Authored: Sun Jul 26 12:06:11 2015 +0530
Committer: James Taylor <jt...@salesforce.com>
Committed: Tue Aug 11 18:36:52 2015 -0700

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/ArrayIT.java     | 201 ++++++++++++++++++-
 .../apache/phoenix/iterate/ExplainTable.java    |   4 +-
 .../phoenix/schema/types/PArrayDataType.java    |   9 +-
 .../phoenix/schema/types/PhoenixArray.java      |   4 -
 4 files changed, 209 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/aa68b556/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayIT.java
index 89997f4..800a7b4 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayIT.java
@@ -2038,5 +2038,204 @@ public class ArrayIT extends BaseClientManagedTimeIT {
         assertFalse(rs.next());
         conn.close();
     }
-    
+
+    @Test
+    public void testComparisonOperatorsForDesc1()throws Exception{
+        long ts = nextTimestamp();
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 10));
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        String ddl = "create table a (k varchar array primary key desc)";
+        conn.createStatement().execute(ddl);
+        conn.close();
+
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 30));
+        conn = DriverManager.getConnection(getUrl(), props);
+        PreparedStatement stmt = conn.prepareStatement("upsert into a values (array['a', 'c'])");
+        stmt.execute();
+        conn.commit();
+        conn.close();
+
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 40));
+        conn = DriverManager.getConnection(getUrl(), props);
+        ResultSet rs;
+        stmt = conn.prepareStatement("select * from a where k >= array['a', 'b']");
+        rs = stmt.executeQuery();
+        assertTrue(rs.next());
+    }
+
+    @Test
+    public void testComparisonOperatorsForDesc2()throws Exception{
+        long ts = nextTimestamp();
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 10));
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        String ddl = "create table a (k varchar array primary key desc)";
+        conn.createStatement().execute(ddl);
+        conn.close();
+
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 30));
+        conn = DriverManager.getConnection(getUrl(), props);
+        PreparedStatement stmt = conn.prepareStatement("upsert into a values (array['a', 'c'])");
+        stmt.execute();
+        conn.commit();
+        conn.close();
+
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 40));
+        conn = DriverManager.getConnection(getUrl(), props);
+        ResultSet rs;
+        stmt = conn.prepareStatement("select * from a where k >= array['a', 'c']");
+        rs = stmt.executeQuery();
+        assertTrue(rs.next());
+    }
+
+    @Test
+    public void testComparisonOperatorsForDesc3()throws Exception{
+        long ts = nextTimestamp();
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 10));
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        String ddl = "create table a (k varchar array primary key desc)";
+        conn.createStatement().execute(ddl);
+        conn.close();
+
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 30));
+        conn = DriverManager.getConnection(getUrl(), props);
+        PreparedStatement stmt = conn.prepareStatement("upsert into a values (array['a', 'c'])");
+        stmt.execute();
+        conn.commit();
+        conn.close();
+
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 40));
+        conn = DriverManager.getConnection(getUrl(), props);
+        ResultSet rs;
+        stmt = conn.prepareStatement("select * from a where k > array['a', 'b']");
+        rs = stmt.executeQuery();
+        assertTrue(rs.next());
+    }
+
+    @Test
+    public void testComparisonOperatorsForDesc4()throws Exception{
+        long ts = nextTimestamp();
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 10));
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        String ddl = "create table a (k varchar array primary key desc)";
+        conn.createStatement().execute(ddl);
+        conn.close();
+
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 30));
+        conn = DriverManager.getConnection(getUrl(), props);
+        PreparedStatement stmt = conn.prepareStatement("upsert into a values (array['a', 'b'])");
+        stmt.execute();
+        conn.commit();
+        conn.close();
+
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 40));
+        conn = DriverManager.getConnection(getUrl(), props);
+        ResultSet rs;
+        stmt = conn.prepareStatement("select * from a where k <= array['a', 'c']");
+        rs = stmt.executeQuery();
+        assertTrue(rs.next());
+    }
+
+    @Test
+    public void testComparisonOperatorsForDesc5()throws Exception{
+        long ts = nextTimestamp();
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 10));
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        String ddl = "create table a (k varchar array primary key desc)";
+        conn.createStatement().execute(ddl);
+        conn.close();
+
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 30));
+        conn = DriverManager.getConnection(getUrl(), props);
+        PreparedStatement stmt = conn.prepareStatement("upsert into a values (array['a', 'b'])");
+        stmt.execute();
+        conn.commit();
+        conn.close();
+
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 40));
+        conn = DriverManager.getConnection(getUrl(), props);
+        ResultSet rs;
+        stmt = conn.prepareStatement("select * from a where k <= array['a', 'b']");
+        rs = stmt.executeQuery();
+        assertTrue(rs.next());
+    }
+
+    @Test
+    public void testComparisonOperatorsForDesc6()throws Exception{
+        long ts = nextTimestamp();
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 10));
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        String ddl = "create table a (k varchar array primary key desc)";
+        conn.createStatement().execute(ddl);
+        conn.close();
+
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 30));
+        conn = DriverManager.getConnection(getUrl(), props);
+        PreparedStatement stmt = conn.prepareStatement("upsert into a values (array['a', 'b'])");
+        stmt.execute();
+        conn.commit();
+        conn.close();
+
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 40));
+        conn = DriverManager.getConnection(getUrl(), props);
+        ResultSet rs;
+        stmt = conn.prepareStatement("select * from a where k < array['a', 'c']");
+        rs = stmt.executeQuery();
+        assertTrue(rs.next());
+    }
+
+    @Test
+    public void testComparisonOperatorsForDesc7()throws Exception{
+        long ts = nextTimestamp();
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 10));
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        String ddl = "create table a (k integer array primary key desc)";
+        conn.createStatement().execute(ddl);
+        conn.close();
+
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 30));
+        conn = DriverManager.getConnection(getUrl(), props);
+        PreparedStatement stmt = conn.prepareStatement("upsert into a values (array[1, 2])");
+        stmt.execute();
+        conn.commit();
+        conn.close();
+
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 40));
+        conn = DriverManager.getConnection(getUrl(), props);
+        ResultSet rs;
+        stmt = conn.prepareStatement("select * from a where k < array[1, 4]");
+        rs = stmt.executeQuery();
+        assertTrue(rs.next());
+    }
+
+    @Test
+    public void testComparisonOperatorsForDesc8()throws Exception{
+        long ts = nextTimestamp();
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 10));
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        String ddl = "create table a (k integer array primary key desc)";
+        conn.createStatement().execute(ddl);
+        conn.close();
+
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 30));
+        conn = DriverManager.getConnection(getUrl(), props);
+        PreparedStatement stmt = conn.prepareStatement("upsert into a values (array[1, 2])");
+        stmt.execute();
+        conn.commit();
+        conn.close();
+
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 40));
+        conn = DriverManager.getConnection(getUrl(), props);
+        ResultSet rs;
+        stmt = conn.prepareStatement("select * from a where k <= array[1, 2]");
+        rs = stmt.executeQuery();
+        assertTrue(rs.next());
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/aa68b556/phoenix-core/src/main/java/org/apache/phoenix/iterate/ExplainTable.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/ExplainTable.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/ExplainTable.java
index 6560308..7b47543 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/ExplainTable.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/ExplainTable.java
@@ -172,7 +172,9 @@ public abstract class ExplainTable {
         SortOrder sortOrder = tableRef.getTable().getPKColumns().get(slotIndex).getSortOrder();
         if (sortOrder == SortOrder.DESC) {
             buf.append('~');
-            range = SortOrder.invert(range, 0, new byte[range.length], 0, range.length);
+            ImmutableBytesWritable ptr = new ImmutableBytesWritable(range);
+            type.coerceBytes(ptr, type, sortOrder, SortOrder.getDefault());
+            range = ptr.get();
         }
         Format formatter = context.getConnection().getFormatter(type);
         buf.append(type.toStringLiteral(range, formatter));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/aa68b556/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataType.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataType.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataType.java
index 6236184..28144fd 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataType.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataType.java
@@ -108,7 +108,8 @@ public abstract class PArrayDataType<T> extends PDataType<T> {
             byteStream = new TrustedByteArrayOutputStream(size + capacity + Bytes.SIZEOF_INT + Bytes.SIZEOF_BYTE
                     + Bytes.SIZEOF_INT);
         } else {
-            int size = arr.getMaxLength() * noOfElements;
+            int elemLength = (arr.getMaxLength() == null ? baseType.getByteSize() : arr.getMaxLength());
+            int size = elemLength * noOfElements;
             // Here the int for noofelements, byte for the version
             byteStream = new TrustedByteArrayOutputStream(size);
         }
@@ -301,8 +302,10 @@ public abstract class PArrayDataType<T> extends PDataType<T> {
             }
             baseType = desiredBaseType;
         } else {
-            pArr = (PhoenixArray)value;
-            pArr = new PhoenixArray(pArr, desiredMaxLength);
+            pArr = (PhoenixArray) value;
+            if (!Objects.equal(maxLength, desiredMaxLength)) {
+                pArr = new PhoenixArray(pArr, desiredMaxLength);
+            }
         }
         ptr.set(toBytes(pArr, baseType, desiredSortOrder, expectedRowKeyOrderOptimizable));
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/aa68b556/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PhoenixArray.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PhoenixArray.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PhoenixArray.java
index 843c831..3dc1050 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PhoenixArray.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PhoenixArray.java
@@ -111,8 +111,6 @@ public class PhoenixArray implements Array,SQLCloseable {
                         }
                     }
     		    }
-		    } else {
-		        maxLength = baseType.getByteSize();
 		    }
 		}
     this.array = convertObjectArrayToPrimitiveArray(elements);
@@ -126,8 +124,6 @@ public class PhoenixArray implements Array,SQLCloseable {
             if (baseType.getByteSize() == null) {
                 elements = coerceToNewLength(baseType, (Object[])pArr.array, desiredMaxLength);
                 maxLength = desiredMaxLength;
-            } else {
-                maxLength = baseType.getByteSize();
             }
         }
         this.array = convertObjectArrayToPrimitiveArray(elements);


[04/12] phoenix git commit: PHOENIX-2155 Add gpg signing to release profile

Posted by ja...@apache.org.
PHOENIX-2155 Add gpg signing to release profile


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

Branch: refs/heads/4.5-HBase-0.98
Commit: 0fcefaad403bc15399b26b9a426cd8fcd082dba1
Parents: 3f03ced
Author: Mujtaba <mu...@apache.org>
Authored: Wed Jul 29 13:06:52 2015 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Tue Aug 11 18:36:52 2015 -0700

----------------------------------------------------------------------
 pom.xml | 15 ++++++++++++++-
 1 file changed, 14 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/0fcefaad/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 0c83b37..0339860 100644
--- a/pom.xml
+++ b/pom.xml
@@ -750,9 +750,22 @@
               </execution>
             </executions>
           </plugin>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-gpg-plugin</artifactId>
+            <version>1.6</version>
+            <executions>
+              <execution>
+              <id>sign-artifacts</id>
+              <phase>verify</phase>
+                <goals>
+                  <goal>sign</goal>
+                </goals>
+              </execution>
+            </executions>
+          </plugin>
         </plugins>
       </build>
     </profile>
-
   </profiles>
 </project>


[10/12] phoenix git commit: PHOENIX-2145 Pherf - Make update stats optional and fix threads not exiting after performance run

Posted by ja...@apache.org.
PHOENIX-2145 Pherf - Make update stats optional and fix threads not exiting after performance run


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

Branch: refs/heads/4.5-HBase-0.98
Commit: 060a00ca50f84da561a7c583f55e22bca5d26641
Parents: 0fcefaa
Author: Mujtaba <mu...@apache.org>
Authored: Tue Aug 4 15:47:44 2015 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Tue Aug 11 18:36:52 2015 -0700

----------------------------------------------------------------------
 phoenix-pherf/pom.xml                           |  3 --
 .../org/apache/phoenix/pherf/DataIngestIT.java  |  4 +-
 .../src/main/assembly/components-minimal.xml    |  2 +-
 .../java/org/apache/phoenix/pherf/Pherf.java    | 23 +++++------
 .../apache/phoenix/pherf/PherfConstants.java    |  5 +++
 .../apache/phoenix/pherf/util/PhoenixUtil.java  | 11 +++++
 .../phoenix/pherf/workload/QueryExecutor.java   |  3 +-
 .../phoenix/pherf/workload/WriteWorkload.java   | 43 ++++++++++----------
 .../org/apache/phoenix/pherf/PherfTest.java     |  2 +-
 9 files changed, 55 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/060a00ca/phoenix-pherf/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-pherf/pom.xml b/phoenix-pherf/pom.xml
index 68a8905..6cfb0e8 100644
--- a/phoenix-pherf/pom.xml
+++ b/phoenix-pherf/pom.xml
@@ -108,9 +108,6 @@
 			</resource>
 			<resource>
 				<directory>config</directory>
-				<includes>
-					<include>**/*.properties</include>
-				</includes>
 			</resource>
 		</resources>
 		<testResources>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/060a00ca/phoenix-pherf/src/it/java/org/apache/phoenix/pherf/DataIngestIT.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/it/java/org/apache/phoenix/pherf/DataIngestIT.java b/phoenix-pherf/src/it/java/org/apache/phoenix/pherf/DataIngestIT.java
index 8a340b3..1defbb1 100644
--- a/phoenix-pherf/src/it/java/org/apache/phoenix/pherf/DataIngestIT.java
+++ b/phoenix-pherf/src/it/java/org/apache/phoenix/pherf/DataIngestIT.java
@@ -20,6 +20,8 @@ package org.apache.phoenix.pherf;
 
 import com.jcabi.jdbc.JdbcSession;
 import com.jcabi.jdbc.Outcome;
+
+import org.apache.phoenix.pherf.PherfConstants.GeneratePhoenixStats;
 import org.apache.phoenix.pherf.configuration.Column;
 import org.apache.phoenix.pherf.configuration.DataModel;
 import org.apache.phoenix.pherf.configuration.DataTypeMapping;
@@ -66,7 +68,7 @@ public class DataIngestIT extends ResultBaseTestIT {
                             scenario.getTableNameWithoutSchemaName(), util.getConnection());
             assertTrue("Could not get phoenix columns.", columnListFromPhoenix.size() > 0);
 
-            WriteWorkload loader = new WriteWorkload(util, parser, scenario);
+            WriteWorkload loader = new WriteWorkload(util, parser, scenario, GeneratePhoenixStats.NO);
             WorkloadExecutor executor = new WorkloadExecutor();
             executor.add(loader);
             executor.get();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/060a00ca/phoenix-pherf/src/main/assembly/components-minimal.xml
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/assembly/components-minimal.xml b/phoenix-pherf/src/main/assembly/components-minimal.xml
index 38c6929..9c9b9a1 100644
--- a/phoenix-pherf/src/main/assembly/components-minimal.xml
+++ b/phoenix-pherf/src/main/assembly/components-minimal.xml
@@ -23,7 +23,7 @@
       <outputDirectory>/</outputDirectory>
       <includes>
         <include>org.apache.phoenix:phoenix-pherf</include>
-        <include>org.jfree:jfreechart</include>
+        <include>org.jfree:*</include>
         <include>com.googlecode.java-diff-utils:diffutils</include>
         <include>org.apache.commons:commons-lang3</include>
         <include>org.apache.commons:commons-math3</include>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/060a00ca/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/Pherf.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/Pherf.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/Pherf.java
index 5a9f45f..0421b6f 100644
--- a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/Pherf.java
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/Pherf.java
@@ -19,6 +19,7 @@
 package org.apache.phoenix.pherf;
 
 import org.apache.commons.cli.*;
+import org.apache.phoenix.pherf.PherfConstants.GeneratePhoenixStats;
 import org.apache.phoenix.pherf.configuration.XMLConfigParser;
 import org.apache.phoenix.pherf.jmx.MonitorManager;
 import org.apache.phoenix.pherf.schema.SchemaReader;
@@ -73,6 +74,8 @@ public class Pherf {
                         + "See pherf.default.dataloader.threadpool in Pherf.properties.");
         options.addOption("h", "help", false, "Get help on using this utility.");
         options.addOption("d", "debug", false, "Put tool in debug mode");
+        options.addOption("stats", false,
+                "Update Phoenix Statistics after data is loaded with -l argument");
     }
 
     private final String zookeeper;
@@ -89,6 +92,7 @@ public class Pherf {
     private final int rowCountOverride;
     private final boolean listFiles;
     private final boolean applySchema;
+    private final GeneratePhoenixStats generateStatistics;
 
     public Pherf(String[] args) throws Exception {
         CommandLineParser parser = new PosixParser();
@@ -126,6 +130,7 @@ public class Pherf {
                 command.hasOption("scenarioFile") ? command.getOptionValue("scenarioFile") : null;
         schemaFile = command.hasOption("schemaFile") ? command.getOptionValue("schemaFile") : null;
         rowCountOverride = Integer.parseInt(command.getOptionValue("rowCountOverride", "0"));
+        generateStatistics = command.hasOption("stats") ? GeneratePhoenixStats.YES : GeneratePhoenixStats.NO;
         String
                 writerThreadPoolSize =
                 command.getOptionValue("writerThreadSize",
@@ -204,23 +209,11 @@ public class Pherf {
             // Schema and Data Load
             if (preLoadData) {
                 logger.info("\nStarting Data Load...");
-                WriteWorkload workload = new WriteWorkload(parser);
+                WriteWorkload workload = new WriteWorkload(parser, generateStatistics);
                 workloadExecutor.add(workload);
 
                 // Wait for dataLoad to complete
                 workloadExecutor.get(workload);
-
-                logger.info("\nGenerate query gold files after data load");
-                QueryExecutor
-                        goldFileGenerator =
-                        new QueryExecutor(parser, phoenixUtil, workloadExecutor.getPool(),
-                                parser.getDataModels(), queryHint, true,
-                                PherfConstants.RunMode.FUNCTIONAL);
-                workloadExecutor
-                        .add(goldFileGenerator);
-
-                // Wait for dataLoad to complete
-                workloadExecutor.get(goldFileGenerator);
             } else {
                 logger.info(
                         "\nSKIPPED: Data Load and schema creation as -l argument not specified");
@@ -254,6 +247,10 @@ public class Pherf {
             if (workloadExecutor != null) {
                 logger.info("Run completed. Shutting down thread pool.");
                 workloadExecutor.shutdown();
+                if (preLoadData) {
+                	System.exit(0);
+                }
+                
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/060a00ca/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 e060e53..12580d4 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
@@ -62,6 +62,11 @@ public class PherfConstants {
     public static final int MONITOR_FREQUENCY = 5000;
     public static final String MONITOR_FILE_NAME = "STATS_MONITOR";
 
+    public static enum GeneratePhoenixStats {
+        YES,
+        NO
+    }
+    
     public static enum RunMode {
         PERFORMANCE,
         FUNCTIONAL

http://git-wip-us.apache.org/repos/asf/phoenix/blob/060a00ca/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/util/PhoenixUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/util/PhoenixUtil.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/util/PhoenixUtil.java
index 5f7d637..5b223b1 100644
--- a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/util/PhoenixUtil.java
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/util/PhoenixUtil.java
@@ -255,4 +255,15 @@ public class PhoenixUtil {
 	public static void setRowCountOverride(int rowCountOverride) {
 		PhoenixUtil.rowCountOverride = rowCountOverride;
 	}
+	
+    /**
+     * Update Phoenix table stats
+     *
+     * @param tableName
+     * @throws Exception
+     */
+    public void updatePhoenixStats(String tableName) throws Exception {
+        logger.info("Updating stats for " + tableName);
+        executeStatement("UPDATE STATISTICS " + tableName);
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/060a00ca/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/QueryExecutor.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/QueryExecutor.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/QueryExecutor.java
index 624188c..4ab76c8 100644
--- a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/QueryExecutor.java
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/QueryExecutor.java
@@ -20,6 +20,7 @@ package org.apache.phoenix.pherf.workload;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.phoenix.pherf.PherfConstants.GeneratePhoenixStats;
 import org.apache.phoenix.pherf.PherfConstants.RunMode;
 import org.apache.phoenix.pherf.configuration.*;
 import org.apache.phoenix.pherf.result.*;
@@ -142,7 +143,7 @@ public class QueryExecutor implements Workload {
                             for (int i = 0; i < writerThreadCount; i++) {
                                 logger.debug("Inserting write workload ( " + i + " ) of ( "
                                         + writerThreadCount + " )");
-                                Workload writes = new WriteWorkload(PhoenixUtil.create(), parser);
+                                Workload writes = new WriteWorkload(PhoenixUtil.create(), parser, GeneratePhoenixStats.NO);
                                 pool.submit(writes.execute());
                             }
                         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/060a00ca/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/WriteWorkload.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/WriteWorkload.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/WriteWorkload.java
index 305521b..b6686c6 100644
--- a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/WriteWorkload.java
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/WriteWorkload.java
@@ -19,6 +19,7 @@
 package org.apache.phoenix.pherf.workload;
 
 import org.apache.phoenix.pherf.PherfConstants;
+import org.apache.phoenix.pherf.PherfConstants.GeneratePhoenixStats;
 import org.apache.phoenix.pherf.configuration.Column;
 import org.apache.phoenix.pherf.configuration.Scenario;
 import org.apache.phoenix.pherf.configuration.WriteParams;
@@ -58,19 +59,24 @@ public class WriteWorkload implements Workload {
 
     private final int threadPoolSize;
     private final int batchSize;
+    private final GeneratePhoenixStats generateStatistics;
 
     public WriteWorkload(XMLConfigParser parser) throws Exception {
-        this(PhoenixUtil.create(), parser);
+        this(PhoenixUtil.create(), parser, GeneratePhoenixStats.NO);
+    }
+    
+    public WriteWorkload(XMLConfigParser parser, GeneratePhoenixStats generateStatistics) throws Exception {
+        this(PhoenixUtil.create(), parser, generateStatistics);
     }
 
-    public WriteWorkload(PhoenixUtil util, XMLConfigParser parser) throws Exception {
-        this(util, parser, null);
+    public WriteWorkload(PhoenixUtil util, XMLConfigParser parser, GeneratePhoenixStats generateStatistics) throws Exception {
+        this(util, parser, null, generateStatistics);
     }
 
-    public WriteWorkload(PhoenixUtil phoenixUtil, XMLConfigParser parser, Scenario scenario)
+    public WriteWorkload(PhoenixUtil phoenixUtil, XMLConfigParser parser, Scenario scenario, GeneratePhoenixStats generateStatistics)
             throws Exception {
         this(phoenixUtil, PherfConstants.create().getProperties(PherfConstants.PHERF_PROPERTIES),
-                parser, scenario);
+                parser, scenario, generateStatistics);
     }
 
     /**
@@ -87,11 +93,12 @@ public class WriteWorkload implements Workload {
      * @throws Exception
      */
     public WriteWorkload(PhoenixUtil phoenixUtil, Properties properties, XMLConfigParser parser,
-            Scenario scenario) throws Exception {
+            Scenario scenario, GeneratePhoenixStats generateStatistics) throws Exception {
         this.pUtil = phoenixUtil;
         this.parser = parser;
         this.rulesApplier = new RulesApplier(parser);
         this.resultUtil = new ResultUtil();
+        this.generateStatistics = generateStatistics;
 
         // Overwrite defaults properties with those given in the configuration. This indicates the
         // scenario is a R/W mixed workload.
@@ -156,9 +163,15 @@ public class WriteWorkload implements Workload {
         List<Future> writeBatches = getBatches(dataLoadThreadTime, scenario);
 
         waitForBatches(dataLoadTimeSummary, scenario, start, writeBatches);
-
-        // always update stats for Phoenix base tables
-        updatePhoenixStats(scenario.getTableName());
+        
+        // Update Phoenix Statistics
+        if (this.generateStatistics == GeneratePhoenixStats.YES) {
+        	logger.info("Updating Phoenix table statistics...");
+        	pUtil.updatePhoenixStats(scenario.getTableName());
+        	logger.info("Stats update done!");
+        } else {
+        	logger.info("Phoenix table stats update not requested.");
+        }
     }
 
     private List<Future> getBatches(DataLoadThreadTime dataLoadThreadTime, Scenario scenario)
@@ -208,18 +221,6 @@ public class WriteWorkload implements Workload {
                 .add(scenario.getTableName(), sumRows, (int) (System.currentTimeMillis() - start));
     }
 
-    /**
-     * TODO Move this method to PhoenixUtil
-     * Update Phoenix table stats
-     *
-     * @param tableName
-     * @throws Exception
-     */
-    public void updatePhoenixStats(String tableName) throws Exception {
-        logger.info("Updating stats for " + tableName);
-        pUtil.executeStatement("UPDATE STATISTICS " + tableName);
-    }
-
     public Future<Info> upsertData(final Scenario scenario, final List<Column> columns,
             final String tableName, final int rowCount,
             final DataLoadThreadTime dataLoadThreadTime) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/060a00ca/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/PherfTest.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/PherfTest.java b/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/PherfTest.java
index 94c8171..731a57a 100644
--- a/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/PherfTest.java
+++ b/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/PherfTest.java
@@ -34,7 +34,7 @@ public class PherfTest {
 
     @Test
     public void testUnknownOption() {
-        String[] args = {"-drop", "all", "-l", "-q", "-m","-bsOption"};
+        String[] args = {"-drop", "all", "-q", "-m","-bsOption"};
 
         // Makes sure that System.exit(1) is called. Release is a required param.
         exit.expectSystemExitWithStatus(1);


[12/12] phoenix git commit: PHOENIX-1791. Adding ability for Pherf Write Workloads to write to a multi-tenant view.

Posted by ja...@apache.org.
PHOENIX-1791. Adding ability for Pherf Write Workloads to write to a multi-tenant view.


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

Branch: refs/heads/4.5-HBase-0.98
Commit: 42476da8357f971995d2504acf421dfc7c22e3a9
Parents: 060a00c
Author: Jan <jf...@salesforce.com>
Authored: Thu Aug 6 11:12:52 2015 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Tue Aug 11 18:36:52 2015 -0700

----------------------------------------------------------------------
 .../org/apache/phoenix/pherf/DataIngestIT.java  | 67 +++++++++++++++++---
 .../phoenix/pherf/configuration/Scenario.java   | 25 ++++++--
 .../pherf/configuration/XMLConfigParser.java    | 23 ++++---
 .../apache/phoenix/pherf/util/PhoenixUtil.java  | 30 +++++----
 .../phoenix/pherf/workload/WriteWorkload.java   | 49 +++++++++-----
 .../datamodel/test_schema_mt_table.sql          | 31 +++++++++
 .../resources/datamodel/test_schema_mt_view.sql |  1 +
 .../test/resources/scenario/test_scenario.xml   | 22 +++++++
 8 files changed, 198 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/42476da8/phoenix-pherf/src/it/java/org/apache/phoenix/pherf/DataIngestIT.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/it/java/org/apache/phoenix/pherf/DataIngestIT.java b/phoenix-pherf/src/it/java/org/apache/phoenix/pherf/DataIngestIT.java
index 1defbb1..297f882 100644
--- a/phoenix-pherf/src/it/java/org/apache/phoenix/pherf/DataIngestIT.java
+++ b/phoenix-pherf/src/it/java/org/apache/phoenix/pherf/DataIngestIT.java
@@ -22,6 +22,19 @@ import com.jcabi.jdbc.JdbcSession;
 import com.jcabi.jdbc.Outcome;
 
 import org.apache.phoenix.pherf.PherfConstants.GeneratePhoenixStats;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
 import org.apache.phoenix.pherf.configuration.Column;
 import org.apache.phoenix.pherf.configuration.DataModel;
 import org.apache.phoenix.pherf.configuration.DataTypeMapping;
@@ -35,15 +48,8 @@ import org.apache.phoenix.pherf.workload.WriteWorkload;
 import org.junit.Before;
 import org.junit.Test;
 
-import java.sql.Connection;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import static org.junit.Assert.*;
+import com.jcabi.jdbc.JdbcSession;
+import com.jcabi.jdbc.Outcome;
 
 public class DataIngestIT extends ResultBaseTestIT {
 
@@ -146,4 +152,47 @@ public class DataIngestIT extends ResultBaseTestIT {
             fail("Failed to load data. An exception was thrown: " + e.getMessage());
         }
     }
+
+
+    @Test
+    /**
+     * Validates that Pherf can write data to a Multi-Tenant View in addition to 
+     * standard Phoenix tables.
+     */
+    public void testMultiTenantViewWriteWorkload() throws Exception {
+        // Arrange
+        Scenario scenario = parser.getScenarioByName("testMTWriteScenario");
+        WorkloadExecutor executor = new WorkloadExecutor();
+        executor.add(new WriteWorkload(util, parser, scenario, GeneratePhoenixStats.NO));
+        
+        // Act
+        try {
+            // Wait for data to load up.
+            executor.get();
+            executor.shutdown();
+        } catch (Exception e) {
+            fail("Failed to load data. An exception was thrown: " + e.getMessage());
+        }
+
+        assertExpectedNumberOfRecordsWritten(scenario);
+    }
+
+    private void assertExpectedNumberOfRecordsWritten(Scenario scenario) throws Exception,
+            SQLException {
+        Connection connection = util.getConnection(scenario.getTenantId());
+        String sql = "select count(*) from " + scenario.getTableName();
+        Integer count = new JdbcSession(connection).sql(sql).select(new Outcome<Integer>() {
+            @Override public Integer handle(ResultSet resultSet, Statement statement)
+                    throws SQLException {
+                while (resultSet.next()) {
+                    return resultSet.getInt(1);
+                }
+                return null;
+            }
+        });
+        assertNotNull("Could not retrieve count. " + count);
+        assertEquals("Expected 100 rows to have been inserted",
+                scenario.getRowCount(), count.intValue());
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/42476da8/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/Scenario.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/Scenario.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/Scenario.java
index 7de96cc..6c949d8 100644
--- a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/Scenario.java
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/Scenario.java
@@ -18,15 +18,16 @@
 
 package org.apache.phoenix.pherf.configuration;
 
-import org.apache.commons.lang.builder.HashCodeBuilder;
-import org.apache.phoenix.pherf.util.PhoenixUtil;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
 
 import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlRootElement;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.phoenix.pherf.util.PhoenixUtil;
 
 @XmlRootElement(namespace = "org.apache.phoenix.pherf.configuration.DataModel")
 public class Scenario {
@@ -37,6 +38,7 @@ public class Scenario {
     private List<QuerySet> querySet = new ArrayList<>();
     private WriteParams writeParams;
     private String name;
+    private String tenantId;
 
     public Scenario() {
         writeParams = new WriteParams();
@@ -162,6 +164,19 @@ public class Scenario {
     public void setName(String name) {
         this.name = name;
     }
+    
+    /**
+     * Tenant Id used by connection of this query
+     * @return
+     */
+    @XmlAttribute
+    public String getTenantId() {
+        return tenantId;
+    }
+
+    public void setTenantId(String tenantId) {
+        this.tenantId = tenantId;
+    }
 
     public WriteParams getWriteParams() {
         return writeParams;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/42476da8/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/XMLConfigParser.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/XMLConfigParser.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/XMLConfigParser.java
index 393fa7e..93dc94c 100644
--- a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/XMLConfigParser.java
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/XMLConfigParser.java
@@ -18,16 +18,6 @@
 
 package org.apache.phoenix.pherf.configuration;
 
-import org.apache.phoenix.pherf.PherfConstants;
-import org.apache.phoenix.pherf.exception.FileLoaderException;
-import org.apache.phoenix.pherf.util.ResourceList;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.xml.bind.JAXBContext;
-import javax.xml.bind.JAXBException;
-import javax.xml.bind.Marshaller;
-import javax.xml.bind.Unmarshaller;
 import java.io.OutputStream;
 import java.nio.file.Path;
 import java.util.ArrayList;
@@ -35,6 +25,17 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.JAXBException;
+import javax.xml.bind.Marshaller;
+import javax.xml.bind.Unmarshaller;
+
+import org.apache.phoenix.pherf.PherfConstants;
+import org.apache.phoenix.pherf.exception.FileLoaderException;
+import org.apache.phoenix.pherf.util.ResourceList;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 public class XMLConfigParser {
 
     private static final Logger logger = LoggerFactory.getLogger(XMLConfigParser.class);
@@ -134,6 +135,8 @@ public class XMLConfigParser {
         if (fullTableName.contains(".")) {
             ret = fullTableName.substring(fullTableName.indexOf(".") + 1, fullTableName.length());
         }
+        // Remove any quotes that may be needed for multi-tenant tables
+        ret = ret.replaceAll("\"", "");
         return ret;
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/42476da8/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/util/PhoenixUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/util/PhoenixUtil.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/util/PhoenixUtil.java
index 5b223b1..19b6bd2 100644
--- a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/util/PhoenixUtil.java
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/util/PhoenixUtil.java
@@ -18,21 +18,29 @@
 
 package org.apache.phoenix.pherf.util;
 
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SCHEM;
+
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+
 import org.apache.phoenix.pherf.PherfConstants;
 import org.apache.phoenix.pherf.configuration.Column;
 import org.apache.phoenix.pherf.configuration.DataTypeMapping;
-
-import java.sql.*;
-import java.util.*;
-
 import org.apache.phoenix.pherf.configuration.Query;
 import org.apache.phoenix.pherf.configuration.QuerySet;
+import org.apache.phoenix.pherf.configuration.Scenario;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SCHEM;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_NAME;
-
 // TODO This class needs to be cleanup up a bit. I just wanted to get an initial placeholder in.
 public class PhoenixUtil {
 	private static final Logger logger = LoggerFactory.getLogger(PhoenixUtil.class);
@@ -80,11 +88,11 @@ public class PhoenixUtil {
         return DriverManager.getConnection(url, props);
     }
 
-    public boolean executeStatement(String sql) throws Exception {
+    public boolean executeStatement(String sql, Scenario scenario) throws Exception {
         Connection connection = null;
         boolean result = false;
         try {
-            connection = getConnection();
+            connection = getConnection(scenario.getTenantId());
             result = executeStatement(sql, connection);
         } finally {
             if (connection != null) {
@@ -262,8 +270,8 @@ public class PhoenixUtil {
      * @param tableName
      * @throws Exception
      */
-    public void updatePhoenixStats(String tableName) throws Exception {
+    public void updatePhoenixStats(String tableName, Scenario scenario) throws Exception {
         logger.info("Updating stats for " + tableName);
-        executeStatement("UPDATE STATISTICS " + tableName);
+        executeStatement("UPDATE STATISTICS " + tableName, scenario);
     }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/42476da8/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/WriteWorkload.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/WriteWorkload.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/WriteWorkload.java
index b6686c6..d0b99af 100644
--- a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/WriteWorkload.java
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/WriteWorkload.java
@@ -18,6 +18,21 @@
 
 package org.apache.phoenix.pherf.workload;
 
+import java.math.BigDecimal;
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.sql.Types;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
 import org.apache.phoenix.pherf.PherfConstants;
 import org.apache.phoenix.pherf.PherfConstants.GeneratePhoenixStats;
 import org.apache.phoenix.pherf.configuration.Column;
@@ -35,17 +50,6 @@ import org.apache.phoenix.pherf.util.RowCalculator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.math.BigDecimal;
-import java.sql.*;
-import java.text.SimpleDateFormat;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Properties;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-
 public class WriteWorkload implements Workload {
     private static final Logger logger = LoggerFactory.getLogger(WriteWorkload.class);
     private final PhoenixUtil pUtil;
@@ -163,15 +167,18 @@ public class WriteWorkload implements Workload {
         List<Future> writeBatches = getBatches(dataLoadThreadTime, scenario);
 
         waitForBatches(dataLoadTimeSummary, scenario, start, writeBatches);
-        
+
         // Update Phoenix Statistics
         if (this.generateStatistics == GeneratePhoenixStats.YES) {
         	logger.info("Updating Phoenix table statistics...");
-        	pUtil.updatePhoenixStats(scenario.getTableName());
+        	pUtil.updatePhoenixStats(scenario.getTableName(), scenario);
         	logger.info("Stats update done!");
         } else {
         	logger.info("Phoenix table stats update not requested.");
         }
+
+        // always update stats for Phoenix base tables
+        updatePhoenixStats(scenario.getTableName(), scenario);
     }
 
     private List<Future> getBatches(DataLoadThreadTime dataLoadThreadTime, Scenario scenario)
@@ -185,7 +192,7 @@ public class WriteWorkload implements Workload {
             List<Column>
                     phxMetaCols =
                     pUtil.getColumnsFromPhoenix(scenario.getSchemaName(),
-                            scenario.getTableNameWithoutSchemaName(), pUtil.getConnection());
+                            scenario.getTableNameWithoutSchemaName(), pUtil.getConnection(scenario.getTenantId()));
             int threadRowCount = rowCalculator.getNext();
             logger.info(
                     "Kick off thread (#" + i + ")for upsert with (" + threadRowCount + ") rows.");
@@ -221,6 +228,18 @@ public class WriteWorkload implements Workload {
                 .add(scenario.getTableName(), sumRows, (int) (System.currentTimeMillis() - start));
     }
 
+    /**
+     * TODO Move this method to PhoenixUtil
+     * Update Phoenix table stats
+     *
+     * @param tableName
+     * @throws Exception
+     */
+    public void updatePhoenixStats(String tableName, Scenario scenario) throws Exception {
+        logger.info("Updating stats for " + tableName);
+        pUtil.executeStatement("UPDATE STATISTICS " + tableName, scenario);
+    }
+
     public Future<Info> upsertData(final Scenario scenario, final List<Column> columns,
             final String tableName, final int rowCount,
             final DataLoadThreadTime dataLoadThreadTime) {
@@ -231,7 +250,7 @@ public class WriteWorkload implements Workload {
                 SimpleDateFormat simpleDateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
                 Connection connection = null;
                 try {
-                    connection = pUtil.getConnection();
+                    connection = pUtil.getConnection(scenario.getTenantId());
                     long logStartTime = System.currentTimeMillis();
                     long
                             maxDuration =

http://git-wip-us.apache.org/repos/asf/phoenix/blob/42476da8/phoenix-pherf/src/test/resources/datamodel/test_schema_mt_table.sql
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/test/resources/datamodel/test_schema_mt_table.sql b/phoenix-pherf/src/test/resources/datamodel/test_schema_mt_table.sql
new file mode 100644
index 0000000..b6791bf
--- /dev/null
+++ b/phoenix-pherf/src/test/resources/datamodel/test_schema_mt_table.sql
@@ -0,0 +1,31 @@
+/*
+  -- 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.
+*/
+CREATE TABLE IF NOT EXISTS PHERF.TEST_MULTI_TENANT_TABLE (
+    TENANT_ID CHAR(15) NOT NULL,
+    IDENTIFIER CHAR(3) NOT NULL,
+    ID CHAR(15) NOT NULL,
+    CREATED_DATE DATE,
+    FIELD VARCHAR,
+    SOME_INT INTEGER
+    CONSTRAINT PK PRIMARY KEY
+    (
+        TENANT_ID,
+        IDENTIFIER,
+        ID
+    )
+) VERSIONS=1,MULTI_TENANT=true

http://git-wip-us.apache.org/repos/asf/phoenix/blob/42476da8/phoenix-pherf/src/test/resources/datamodel/test_schema_mt_view.sql
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/test/resources/datamodel/test_schema_mt_view.sql b/phoenix-pherf/src/test/resources/datamodel/test_schema_mt_view.sql
new file mode 100644
index 0000000..5f5d7ec
--- /dev/null
+++ b/phoenix-pherf/src/test/resources/datamodel/test_schema_mt_view.sql
@@ -0,0 +1 @@
+CREATE VIEW IF NOT EXISTS PHERF.TEST_VIEW (field1 VARCHAR, field2 VARCHAR) AS SELECT * FROM PHERF.TEST_MULTI_TENANT_TABLE
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/42476da8/phoenix-pherf/src/test/resources/scenario/test_scenario.xml
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/test/resources/scenario/test_scenario.xml b/phoenix-pherf/src/test/resources/scenario/test_scenario.xml
index fddf022..b5fe564 100644
--- a/phoenix-pherf/src/test/resources/scenario/test_scenario.xml
+++ b/phoenix-pherf/src/test/resources/scenario/test_scenario.xml
@@ -127,6 +127,25 @@
             <name>NEWVAL_STRING</name>
             <prefix>TSTPRFX</prefix>
         </column>
+        <column>
+            <type>CHAR</type>
+            <length>3</length>
+            <userDefined>true</userDefined>
+            <dataSequence>LIST</dataSequence>
+            <name>IDENTIFIER</name>
+            <valuelist>
+                <!-- Distributes according to specified values. These must total 100 -->
+                <datavalue distribution="60">
+                    <value>ABC</value>
+                </datavalue>
+                <datavalue distribution="20">
+                    <value>XYZ</value>
+                </datavalue>
+                <datavalue distribution="20">
+                    <value>LMN</value>
+                </datavalue>
+            </valuelist>            
+        </column>        
     </datamapping>
     <scenarios>
         <scenario tableName="PHERF.TEST_TABLE" rowCount="100" name="testScenarioRW">
@@ -201,5 +220,8 @@
                 <query id="q4" statement="select sum(SOME_INT) from PHERF.TEST_TABLE"/>
             </querySet>
         </scenario>
+        <!-- Test writing to a Multi-tenant View -->
+        <scenario tableName="PHERF.TEST_VIEW" tenantId="abcdefghijklmno" rowCount="100" name="testMTWriteScenario">
+        </scenario>
     </scenarios>
 </datamodel>
\ No newline at end of file