You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by co...@apache.org on 2015/04/27 19:21:18 UTC

phoenix git commit: PHOENIX-1728 - Pherf - Make tests use mini cluster. Port from master.

Repository: phoenix
Updated Branches:
  refs/heads/4.4-HBase-1.0 77c5ba76e -> 2a0dc609f


PHOENIX-1728 - Pherf - Make tests use mini cluster. Port from master.


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

Branch: refs/heads/4.4-HBase-1.0
Commit: 2a0dc609faf08af8c9e900cf455bd4505c7d2c8c
Parents: 77c5ba7
Author: cmarcel <cm...@salesforce.com>
Authored: Mon Apr 27 10:18:47 2015 -0700
Committer: cmarcel <cm...@salesforce.com>
Committed: Mon Apr 27 10:18:47 2015 -0700

----------------------------------------------------------------------
 phoenix-pherf/pom.xml                           | 143 ++++++++++++++-----
 .../java/org/apache/phoenix/pherf/Pherf.java    |   1 -
 .../phoenix/pherf/loaddata/DataLoader.java      |  14 +-
 .../phoenix/pherf/schema/SchemaReader.java      |   7 +-
 .../apache/phoenix/pherf/util/PhoenixUtil.java  |  48 ++++---
 .../apache/phoenix/pherf/util/ResourceList.java |   8 +-
 .../pherf/workload/WorkloadExecutor.java        |   4 +-
 .../datamodel/create_prod_test_unsalted.sql     |   2 +-
 .../scenario/prod_test_unsalted_scenario.xml    |   4 +-
 .../phoenix/pherf/BaseTestWithCluster.java      |  45 ------
 .../phoenix/pherf/ConfigurationParserTest.java  |   2 +-
 .../apache/phoenix/pherf/DataIngestTest.java    |  78 ----------
 .../apache/phoenix/pherf/DataLoaderTest.java    |  94 ------------
 .../org/apache/phoenix/pherf/PherfTest.java     |  26 +---
 .../org/apache/phoenix/pherf/ResultTest.java    |  12 --
 .../apache/phoenix/pherf/RuleGeneratorTest.java |   7 +-
 .../apache/phoenix/pherf/SchemaReaderTest.java  |  73 ----------
 .../apache/phoenix/pherf/TestHBaseProps.java    |   1 -
 .../test/resources/datamodel/test_schema.sql    |  33 ++---
 .../test/resources/scenario/test_scenario.xml   |   4 +-
 pom.xml                                         |   2 +-
 21 files changed, 179 insertions(+), 429 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a0dc609/phoenix-pherf/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-pherf/pom.xml b/phoenix-pherf/pom.xml
index dad3297..0901f71 100644
--- a/phoenix-pherf/pom.xml
+++ b/phoenix-pherf/pom.xml
@@ -25,9 +25,8 @@
         <version>4.4.0-SNAPSHOT</version>
     </parent>
 
-    <artifactId>pherf</artifactId>
+    <artifactId>phoenix-pherf</artifactId>
     <packaging>jar</packaging>
-    <!--<version>1.0-SNAPSHOT</version>-->
     <name>Phoenix - Pherf</name>
 
     <properties>
@@ -57,23 +56,13 @@
                     <plugin>
                         <groupId>org.apache.maven.plugins</groupId>
                         <artifactId>maven-surefire-plugin</artifactId>
-                        <configuration>
-                            <!-- Some tests require a cluster to be up. Disable them by default until we fix that problem -->
-                            <skipTests>true</skipTests>
-                            <systemPropertyVariables>
-                                <java.util.logging.config.file>${logging.location}</java.util.logging.config.file>
-                                <app.home>${project.basedir}</app.home>
-                            </systemPropertyVariables>
-                        </configuration>
                     </plugin>
                     <plugin>
                         <groupId>org.apache.maven.plugins</groupId>
                         <artifactId>maven-resources-plugin</artifactId>
-                        <version>2.4</version>
                     </plugin>
                     <plugin>
                         <artifactId>maven-assembly-plugin</artifactId>
-                        <version>2.4</version>
                         <executions>
                             <execution>
                                 <id>make-dependency-jar</id>
@@ -134,23 +123,13 @@
                     <plugin>
                         <groupId>org.apache.maven.plugins</groupId>
                         <artifactId>maven-surefire-plugin</artifactId>
-                        <configuration>
-                            <!-- Some tests require a cluster to be up. Disable them by default until we fix that problem -->
-                            <skipTests>true</skipTests>
-                            <systemPropertyVariables>
-                                <java.util.logging.config.file>${logging.location}</java.util.logging.config.file>
-                                <app.home>${project.basedir}</app.home>
-                            </systemPropertyVariables>
-                        </configuration>
                     </plugin>
                     <plugin>
                         <groupId>org.apache.maven.plugins</groupId>
                         <artifactId>maven-resources-plugin</artifactId>
-                        <version>2.4</version>
                     </plugin>
                     <plugin>
                         <artifactId>maven-assembly-plugin</artifactId>
-                        <version>2.4</version>
                         <executions>
                             <execution>
                                 <id>make-dependency-jar</id>
@@ -231,6 +210,28 @@
             <version>3.3</version>
         </dependency>
         <dependency>
+            <groupId>org.jfree</groupId>
+            <artifactId>jfreechart</artifactId>
+            <version>1.0.19</version>
+        </dependency>
+        <dependency>
+            <groupId>joda-time</groupId>
+            <artifactId>joda-time</artifactId>
+            <version>1.6.2</version>
+        </dependency>
+        <dependency>
+            <groupId>com.googlecode.java-diff-utils</groupId>
+            <artifactId>diffutils</artifactId>
+            <version>1.2.1</version>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+            <version>1.7.10</version>
+        </dependency>
+
+        <!-- Test Dependencies -->
+        <dependency>
             <groupId>junit</groupId>
             <artifactId>junit</artifactId>
             <version>4.11</version>
@@ -242,41 +243,107 @@
             <version>1.0.0.0</version>
             <scope>test</scope>
         </dependency>
-
         <dependency>
             <groupId>com.github.stefanbirkner</groupId>
             <artifactId>system-rules</artifactId>
             <version>1.8.0</version>
         </dependency>
         <dependency>
-            <groupId>log4j</groupId>
-            <artifactId>log4j</artifactId>
-            <version>1.2.17</version>
+            <groupId>org.apache.hbase</groupId>
+            <artifactId>hbase-testing-util</artifactId>
+            <version>${hbase.version}</version>
+            <scope>test</scope>
+            <optional>true</optional>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.jruby</groupId>
+                    <artifactId>jruby-complete</artifactId>
+                </exclusion>
+            </exclusions>
         </dependency>
         <dependency>
-            <groupId>org.jfree</groupId>
-            <artifactId>jfreechart</artifactId>
-            <version>1.0.19</version>
+            <groupId>org.apache.hbase</groupId>
+            <artifactId>hbase-it</artifactId>
+            <version>${hbase.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.jruby</groupId>
+                    <artifactId>jruby-complete</artifactId>
+                </exclusion>
+            </exclusions>
         </dependency>
         <dependency>
-            <groupId>joda-time</groupId>
-            <artifactId>joda-time</artifactId>
-            <version>1.6.2</version>
+            <groupId>org.apache.hbase</groupId>
+            <artifactId>hbase-common</artifactId>
+            <version>${hbase.version}</version>
         </dependency>
         <dependency>
-            <groupId>com.googlecode.java-diff-utils</groupId>
-            <artifactId>diffutils</artifactId>
-            <version>1.2.1</version>
+            <groupId>org.apache.hbase</groupId>
+            <artifactId>hbase-protocol</artifactId>
+            <version>${hbase.version}</version>
         </dependency>
         <dependency>
-            <groupId>org.slf4j</groupId>
-            <artifactId>slf4j-log4j12</artifactId>
-            <version>1.7.10</version>
+            <groupId>org.apache.hbase</groupId>
+            <artifactId>hbase-client</artifactId>
+            <version>${hbase.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hbase</groupId>
+            <artifactId>hbase-hadoop-compat</artifactId>
+            <version>${hbase.version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hbase</groupId>
+            <artifactId>hbase-hadoop-compat</artifactId>
+            <version>${hbase.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hbase</groupId>
+            <artifactId>hbase-hadoop2-compat</artifactId>
+            <version>${hbase.version}</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hbase</groupId>
+            <artifactId>hbase-hadoop2-compat</artifactId>
+            <version>${hbase.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-common</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-mapreduce-client-core</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-minicluster</artifactId>
         </dependency>
     </dependencies>
     <build>
         <plugins>
             <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>build-helper-maven-plugin</artifactId>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-failsafe-plugin</artifactId>
+            </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>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a0dc609/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 ac386f2..0ccc49a 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
@@ -116,7 +116,6 @@ public class Pherf {
         }
         PhoenixUtil.setZookeeper(zookeeper);
         PhoenixUtil.setRowCountOverride(rowCountOverride);
-        PhoenixUtil.writeSfdcClientProperty();
     }
 
     public static void main(String[] args) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a0dc609/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/loaddata/DataLoader.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/loaddata/DataLoader.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/loaddata/DataLoader.java
index bb7a929..abec1b4 100644
--- a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/loaddata/DataLoader.java
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/loaddata/DataLoader.java
@@ -53,7 +53,7 @@ import org.apache.phoenix.pherf.util.PhoenixUtil;
 
 public class DataLoader {
     private static final Logger logger = LoggerFactory.getLogger(DataLoader.class);
-    private final PhoenixUtil pUtil = new PhoenixUtil();
+    private final PhoenixUtil pUtil;
     private final XMLConfigParser parser;
     private final RulesApplier rulesApplier;
     private final ResultUtil resultUtil;
@@ -63,7 +63,11 @@ public class DataLoader {
     private final int batchSize;
 
     public DataLoader(XMLConfigParser parser) throws Exception {
-        this(new ResourceList().getProperties(), parser);
+        this(new PhoenixUtil(), parser);
+    }
+
+    public DataLoader(PhoenixUtil phoenixUtil, XMLConfigParser parser) throws Exception{
+        this(phoenixUtil, new ResourceList().getProperties(), parser);
     }
 
     /**
@@ -72,9 +76,10 @@ public class DataLoader {
      * @param parser
      * @throws Exception
      */
-    public DataLoader(Properties properties, XMLConfigParser parser) throws Exception {
+    public DataLoader(PhoenixUtil phoenixUtil, Properties properties, XMLConfigParser parser) throws Exception {
+        this.pUtil = phoenixUtil;
         this.parser = parser;
-        this.rulesApplier = new RulesApplier(this.parser);
+        this.rulesApplier = new RulesApplier(parser);
         this.resultUtil = new ResultUtil();
         int size = Integer.parseInt(properties.getProperty("pherf.default.dataloader.threadpool"));
         this.threadPoolSize = (size == 0) ? Runtime.getRuntime().availableProcessors() : size;
@@ -85,7 +90,6 @@ public class DataLoader {
 
     public void execute() throws Exception {
         try {
-            DataModel model = getParser().getDataModels().get(0);
             DataLoadTimeSummary dataLoadTimeSummary = new DataLoadTimeSummary();
             DataLoadThreadTime dataLoadThreadTime = new DataLoadThreadTime();
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a0dc609/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/schema/SchemaReader.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/schema/SchemaReader.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/schema/SchemaReader.java
index 234dd14..4761211 100644
--- a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/schema/SchemaReader.java
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/schema/SchemaReader.java
@@ -34,7 +34,7 @@ import java.util.Collection;
 
 public class SchemaReader {
     private static final Logger logger = LoggerFactory.getLogger(SchemaReader.class);
-    private final PhoenixUtil pUtil = new PhoenixUtil();
+    private final PhoenixUtil pUtil;
     private Collection<Path> resourceList;
     private final String searchPattern;
     private final ResourceList resourceUtil;
@@ -45,6 +45,11 @@ public class SchemaReader {
      * @throws Exception
      */
     public SchemaReader(final String searchPattern) throws Exception {
+        this(new PhoenixUtil(), searchPattern);
+    }
+
+    public SchemaReader(PhoenixUtil util, final String searchPattern) throws Exception {
+        this.pUtil = util;
         this.searchPattern = searchPattern;
         this.resourceUtil = new ResourceList(PherfConstants.RESOURCE_DATAMODEL);
         read();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a0dc609/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 0959700..83e324d 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
@@ -38,36 +38,38 @@ public class PhoenixUtil {
 	private static final Logger logger = LoggerFactory.getLogger(PhoenixUtil.class);
 	private static String zookeeper;
 	private static int rowCountOverride = 0;
-	
+    private boolean testEnabled;
+
+    public PhoenixUtil() {
+        this(false);
+    }
+
+    public PhoenixUtil(final boolean testEnabled) {
+        this.testEnabled = testEnabled;
+    }
+
     public Connection getConnection() throws Exception{
     	return getConnection(null);
     }
 	
-    public Connection getConnection(String tenantId) throws Exception{
-		if (null == zookeeper) {
-			throw new IllegalArgumentException("Zookeeper must be set before initializing connection!");
-		}
-    	Properties props = new Properties();
-    	if (null != tenantId) {
-    		props.setProperty("TenantId", tenantId);
-   			logger.debug("\nSetting tenantId to " + tenantId);
-    	}
-    	Connection connection = DriverManager.getConnection("jdbc:phoenix:" + zookeeper, props);
-        return connection;
+    public Connection getConnection(String tenantId) throws Exception {
+        return getConnection(tenantId, testEnabled);
     }
 
-    public static void writeSfdcClientProperty() throws IOException {
-		Configuration conf = HBaseConfiguration.create();
-		Map<String, String> sfdcProperty = conf.getValByRegex("sfdc");
-    	Properties props = new Properties();
-		for (Map.Entry<String, String> entry : sfdcProperty.entrySet()) {
-			props.put(entry.getKey(), entry.getValue());
-			logger.debug("\nSetting sfdc connection property " + entry.getKey() + " to " + entry.getValue());
-		}
-        OutputStream out = new java.io.FileOutputStream(new File("sfdc-hbase-client.properties"));
-        props.store(out,"client properties");
+    public Connection getConnection(String tenantId, boolean testEnabled) throws Exception {
+        if (null == zookeeper) {
+            throw new IllegalArgumentException(
+                    "Zookeeper must be set before initializing connection!");
+        }
+        Properties props = new Properties();
+        if (null != tenantId) {
+            props.setProperty("TenantId", tenantId);
+            logger.debug("\nSetting tenantId to " + tenantId);
+        }
+        String url = "jdbc:phoenix:" + zookeeper + (testEnabled ? ";test=true" : "");
+        return DriverManager.getConnection(url, props);
     }
- 
+
     public boolean executeStatement(String sql) throws Exception {
         Connection connection = null;
         boolean result = false;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a0dc609/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/util/ResourceList.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/util/ResourceList.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/util/ResourceList.java
index 0677dae..ac1e74d 100644
--- a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/util/ResourceList.java
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/util/ResourceList.java
@@ -163,11 +163,11 @@ public class ResourceList {
             final ZipEntry ze = (ZipEntry) e.nextElement();
             final String fileName = ze.getName();
             final boolean accept = pattern.matcher(fileName).matches();
-            logger.debug("fileName:" + fileName);
-            logger.debug("File:" + file.toString());
-            logger.debug("Match:" + accept);
+            logger.trace("fileName:" + fileName);
+            logger.trace("File:" + file.toString());
+            logger.trace("Match:" + accept);
             if (accept) {
-                logger.debug("Adding File from Jar: " + fileName);
+                logger.trace("Adding File from Jar: " + fileName);
                 retVal.add("/" + fileName);
             }
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a0dc609/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/WorkloadExecutor.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/WorkloadExecutor.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/WorkloadExecutor.java
index 5cc6515..0509bdc 100644
--- a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/WorkloadExecutor.java
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/WorkloadExecutor.java
@@ -39,7 +39,6 @@ public class WorkloadExecutor {
     private final XMLConfigParser parser;
     private MonitorManager monitor;
     private Future monitorThread;
-    private final Properties properties;
     private final int poolSize;
 
     private final ExecutorService pool;
@@ -61,7 +60,6 @@ public class WorkloadExecutor {
 
     public WorkloadExecutor(Properties properties, XMLConfigParser parser, boolean monitor) throws Exception {
         this.parser = parser;
-        this.properties = properties;
         this.poolSize = (properties.getProperty("pherf.default.threadpool") == null)
                 ? PherfConstants.DEFAULT_THREAD_POOL_SIZE
                 : Integer.parseInt(properties.getProperty("pherf.default.threadpool"));
@@ -79,7 +77,7 @@ public class WorkloadExecutor {
      */
     public void executeDataLoad() throws Exception {
         logger.info("\n\nStarting Data Loader...");
-        DataLoader dataLoader = new DataLoader(properties, parser);
+        DataLoader dataLoader = new DataLoader(parser);
         dataLoader.execute();
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a0dc609/phoenix-pherf/src/main/resources/datamodel/create_prod_test_unsalted.sql
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/resources/datamodel/create_prod_test_unsalted.sql b/phoenix-pherf/src/main/resources/datamodel/create_prod_test_unsalted.sql
index ec2f88d..dd1e2d8 100644
--- a/phoenix-pherf/src/main/resources/datamodel/create_prod_test_unsalted.sql
+++ b/phoenix-pherf/src/main/resources/datamodel/create_prod_test_unsalted.sql
@@ -28,6 +28,6 @@ CREATE TABLE IF NOT EXISTS PHERF.PHERF_PROD_TEST_UNSALTED (
     CONSTRAINT PK PRIMARY KEY 
     (
         TENANT_ID,
-        CREATED_DATE DESC,
+        CREATED_DATE DESC
     )
 ) VERSIONS=1,MULTI_TENANT=true

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a0dc609/phoenix-pherf/src/main/resources/scenario/prod_test_unsalted_scenario.xml
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/resources/scenario/prod_test_unsalted_scenario.xml b/phoenix-pherf/src/main/resources/scenario/prod_test_unsalted_scenario.xml
index 5e20bf0..9514089 100644
--- a/phoenix-pherf/src/main/resources/scenario/prod_test_unsalted_scenario.xml
+++ b/phoenix-pherf/src/main/resources/scenario/prod_test_unsalted_scenario.xml
@@ -17,7 +17,7 @@
   ~   limitations under the License.
   -->
 
-<datamodel release="194" name="PROD_TEST_UNSALTED">
+<datamodel name="PROD_TEST_UNSALTED">
     <datamapping>
         <column>
             <!-- This column type defines what will generally happen to VARCHAR fields unless they are explicitly defined or overridden elsewhere -->
@@ -304,7 +304,7 @@
         </column>
     </datamapping>
     <scenarios>
-        <scenario tableName="PHERF.PHERF_PROD_TEST_UNSALTED" rowCount="10000">
+        <scenario tableName="PHERF.PHERF_PROD_TEST_UNSALTED" rowCount="10">
             <!-- Scenario level rule overrides will be unsupported in V1.
                     You can use the general datamappings in the mean time-->
             <dataOverride>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a0dc609/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/BaseTestWithCluster.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/BaseTestWithCluster.java b/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/BaseTestWithCluster.java
deleted file mode 100644
index 8e33a15..0000000
--- a/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/BaseTestWithCluster.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- *   or more contributor license agreements.  See the NOTICE file
- *   distributed with this work for additional information
- *   regarding copyright ownership.  The ASF licenses this file
- *   to you under the Apache License, Version 2.0 (the
- *   "License"); you may not use this file except in compliance
- *   with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- *   Unless required by applicable law or agreed to in writing, software
- *   distributed under the License is distributed on an "AS IS" BASIS,
- *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *   See the License for the specific language governing permissions and
- *   limitations under the License.
- */
-
-package org.apache.phoenix.pherf;
-
-import org.apache.phoenix.pherf.util.PhoenixUtil;
-import org.junit.BeforeClass;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.net.URISyntaxException;
-import java.net.URL;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-
-import static org.junit.Assert.assertNotNull;
-
-public class BaseTestWithCluster {
-    static final String matcherScenario = PherfConstants.SCENARIO_ROOT_PATTERN + ".xml";
-    private static final Logger logger = LoggerFactory.getLogger(BaseTestWithCluster.class);
-    protected static PhoenixUtil util;
-
-    @BeforeClass
-    public static void initQuorum() {
-        util = new PhoenixUtil();
-        String zookeeper = ((System.getProperty("ZK_QUORUM") == null) || System.getProperty("ZK_QUORUM").equals("")) ? "localhost" : System.getProperty("ZK_QUORUM");
-        PhoenixUtil.setZookeeper(zookeeper);
-        logger.info("Using quorum:" + zookeeper);
-    }
-}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a0dc609/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ConfigurationParserTest.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ConfigurationParserTest.java b/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ConfigurationParserTest.java
index 080c0f3..fcc353e 100644
--- a/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ConfigurationParserTest.java
+++ b/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ConfigurationParserTest.java
@@ -84,7 +84,7 @@ public class ConfigurationParserTest {
             Scenario scenario = scenarioList.get(0);
             assertNotNull(scenario);
             assertEquals("PHERF.TEST_TABLE", scenario.getTableName());
-            assertEquals(50, scenario.getRowCount());
+            assertEquals(10, scenario.getRowCount());
             assertEquals(1, scenario.getDataOverride().getColumn().size());
             QuerySet qs = scenario.getQuerySet().get(0);
             assertEquals(ExecutionType.SERIAL, qs.getExecutionType());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a0dc609/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/DataIngestTest.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/DataIngestTest.java b/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/DataIngestTest.java
deleted file mode 100644
index 0fc0dd9..0000000
--- a/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/DataIngestTest.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- *   or more contributor license agreements.  See the NOTICE file
- *   distributed with this work for additional information
- *   regarding copyright ownership.  The ASF licenses this file
- *   to you under the Apache License, Version 2.0 (the
- *   "License"); you may not use this file except in compliance
- *   with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- *   Unless required by applicable law or agreed to in writing, software
- *   distributed under the License is distributed on an "AS IS" BASIS,
- *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *   See the License for the specific language governing permissions and
- *   limitations under the License.
- */
-
-package org.apache.phoenix.pherf;
-
-import org.apache.phoenix.pherf.configuration.Column;
-import org.apache.phoenix.pherf.configuration.DataTypeMapping;
-import org.apache.phoenix.pherf.configuration.Scenario;
-import org.apache.phoenix.pherf.configuration.XMLConfigParser;
-import org.apache.phoenix.pherf.loaddata.DataLoader;
-import org.apache.phoenix.pherf.rules.DataValue;
-import org.apache.phoenix.pherf.rules.RulesApplier;
-import org.apache.phoenix.pherf.schema.SchemaReader;
-import org.junit.Test;
-
-import java.nio.file.Path;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-
-public class DataIngestTest extends BaseTestWithCluster {
-    static final String matcherScenario = ".*scenario/.*test.*xml";
-    static final String matcherSchema = ".*datamodel/.*test.*sql";
-
-    @Test
-    public void generateData() throws Exception {
-        SchemaReader reader = new SchemaReader(matcherSchema);
-        XMLConfigParser parser = new XMLConfigParser(matcherScenario);
-
-        // 1. Generate table schema from file
-        List<Path> resources = new ArrayList<>(reader.getResourceList());
-        assertTrue("Could not pull list of schema files.", resources.size() > 0);
-        assertNotNull("Could not read schema file.", reader.resourceToString(resources.get(0)));
-        reader.applySchema();
-
-        // 2. Load the metadata of for the test tables
-        Scenario scenario = parser.getScenarios().get(0);
-        List<Column> columnListFromPhoenix = util.getColumnsFromPhoenix(scenario.getSchemaName(), scenario.getTableNameWithoutSchemaName(), util.getConnection());
-        assertTrue("Could not get phoenix columns.", columnListFromPhoenix.size() > 0);
-        DataLoader loader = new DataLoader(parser);
-        RulesApplier rulesApplier = loader.getRulesApplier();
-        List<Map> modelList = rulesApplier.getModelList();
-        assertTrue("Could not generate the modelList", modelList.size() > 0);
-
-        for (Column column : columnListFromPhoenix) {
-            DataValue data = rulesApplier.getDataForRule(scenario, column);
-
-            // We are generating data values so the value should have been specified by this point.
-            assertTrue("Failed to retrieve data for column type: " + column.getType(), data != null);
-
-            // Test that we still retrieve the GENERAL_CHAR rule even after an override is applied to another CHAR type.
-            // FIELD_HISTORY_ARCHIVE_ID Column does not  specify an override so we should get the default rule.
-            if ((column.getType() == DataTypeMapping.CHAR) && (column.getName().equals("FIELD_HISTORY_ARCHIVE_ID"))) {
-                assertTrue("Failed to retrieve data for column type: ", data.getDistribution() == Integer.MIN_VALUE);
-            }
-        }
-
-        loader.execute();
-    }
-}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a0dc609/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/DataLoaderTest.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/DataLoaderTest.java b/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/DataLoaderTest.java
deleted file mode 100644
index f4d676e..0000000
--- a/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/DataLoaderTest.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- *   or more contributor license agreements.  See the NOTICE file
- *   distributed with this work for additional information
- *   regarding copyright ownership.  The ASF licenses this file
- *   to you under the Apache License, Version 2.0 (the
- *   "License"); you may not use this file except in compliance
- *   with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- *   Unless required by applicable law or agreed to in writing, software
- *   distributed under the License is distributed on an "AS IS" BASIS,
- *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *   See the License for the specific language governing permissions and
- *   limitations under the License.
- */
-
-package org.apache.phoenix.pherf;
-
-import org.apache.phoenix.pherf.configuration.DataModel;
-import org.apache.phoenix.pherf.configuration.XMLConfigParser;
-import org.apache.phoenix.pherf.loaddata.DataLoader;
-import org.apache.phoenix.pherf.util.RowCalculator;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import static junit.framework.Assert.assertEquals;
-import static junit.framework.Assert.fail;
-
-public class DataLoaderTest extends BaseTestWithCluster {
-
-    /**
-     * Test rows divide evenly with large rows and small threadpool
-     * @throws Exception
-     */
-    @Test
-    public void testRowsEvenDivide() throws Exception {
-        int threadPoolSize = 10;
-        int tableRowCount = 100;
-        assertRowsSum(threadPoolSize, tableRowCount);
-    }
-
-    /**
-     * Test rows add up when not divided evenly with large rows and small threadpool
-     *
-     * @throws Exception
-     */
-    @Test
-    public void testRowsNotEvenDivide() throws Exception {
-        int threadPoolSize = 9;
-        int tableRowCount = 100;
-        assertRowsSum(threadPoolSize, tableRowCount);
-    }
-
-    /**
-     * Test rows add up when not divided evenly with large threadpool and small rowcount
-     *
-     * @throws Exception
-     */
-    @Test
-    public void testRowsNotEvenDivideSmallRC() throws Exception {
-        int threadPoolSize = 50;
-        int tableRowCount = 21;
-        assertRowsSum(threadPoolSize, tableRowCount);
-    }
-
-    /**
-     * Test rows count equal to thread pool
-     *
-     * @throws Exception
-     */
-    @Test
-    public void testRowsEqualToPool() throws Exception {
-        int threadPoolSize = 50;
-        int tableRowCount = 50;
-        assertRowsSum(threadPoolSize, tableRowCount);
-    }
-
-    private void assertRowsSum(int threadPoolSize, int tableRowCount) {
-        int sum = 0;
-        RowCalculator rc = new RowCalculator(threadPoolSize, tableRowCount);
-        assertEquals("Rows generated did not match expected count! ", threadPoolSize, rc.size());
-
-        // Sum of all rows should equal expected row count
-        for (int i = 0; i < threadPoolSize; i++) {
-            sum += rc.getNext();
-        }
-        assertEquals("Rows did not sum up correctly", tableRowCount, sum);
-
-        // Ensure rows were removed from list
-        assertEquals(rc.size(), 0);
-    }
-}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a0dc609/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 f15b4d4..94c8171 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
@@ -22,41 +22,17 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.contrib.java.lang.system.ExpectedSystemExit;
 
-import java.sql.Date;
-
-public class PherfTest extends BaseTestWithCluster {
+public class PherfTest {
     @Rule
     public final ExpectedSystemExit exit = ExpectedSystemExit.none();
 
     @Test
-    public void testPherfMain() {
-        String[] args = {"-drop", "all", "-l", "-q", "-m",
-                "--monitorFrequency", "100",
-                "-z", "localhost",
-                "--scenarioFile", ".*user_defined_scenario.xml",
-                "--schemaFile", ".*user_defined_schema_194.sql"};
-        Pherf.main(args);
-    }
-
-    @Test
     public void testListArgument() {
         String[] args = {"-listFiles"};
         Pherf.main(args);
     }
 
     @Test
-    public void testReleaseExists() {
-        String[] args = {"-drop", "all", "-l", "-q", "-m",
-                "--monitorFrequency", "100",
-                "--scenarioFile", ".*test_scenario.xml",
-                "--schemaFile", ".*user_defined_schema_194.sql"};
-
-        // Makes sure that System.exit(1) is called. Release is a required param.
-        exit.expectSystemExitWithStatus(1);
-        Pherf.main(args);
-    }
-
-    @Test
     public void testUnknownOption() {
         String[] args = {"-drop", "all", "-l", "-q", "-m","-bsOption"};
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a0dc609/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ResultTest.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ResultTest.java b/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ResultTest.java
index 8152390..0f4dfd1 100644
--- a/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ResultTest.java
+++ b/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ResultTest.java
@@ -25,7 +25,6 @@ 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.RunMode;
 import org.apache.phoenix.pherf.jmx.MonitorManager;
 
@@ -41,17 +40,6 @@ import org.apache.phoenix.pherf.configuration.Query;
 public class ResultTest {
 
     @Test
-    public void testWriters() throws Exception {
-        ExecutorService executorService = Executors.newFixedThreadPool(1);
-        List<ResultHandler> writers = new ArrayList<>();
-//        Monitor monitor = new Monitor(new XMLConfigParser("test.*xml"), 100);
-//        Future future = executorService.submit(monitor);
-
-        writers.add(new CSVResultHandler(PherfConstants.MONITOR_FILE_NAME, ResultFileDetails.CSV_DETAILED_PERFORMANCE));
-
-    }
-
-    @Test
     public void testMonitorWriter() throws Exception {
         String[] row = "org.apache.phoenix.pherf:type=PherfWriteThreads,6,Mon Jan 05 15:14:00 PST 2015".split(PherfConstants.RESULT_FILE_DELIMETER);
         ResultHandler resultMonitorWriter = null;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a0dc609/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/RuleGeneratorTest.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/RuleGeneratorTest.java b/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/RuleGeneratorTest.java
index 8eafd56..15d4608 100644
--- a/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/RuleGeneratorTest.java
+++ b/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/RuleGeneratorTest.java
@@ -22,6 +22,7 @@ import org.apache.phoenix.pherf.configuration.*;
 import org.apache.phoenix.pherf.loaddata.DataLoader;
 import org.apache.phoenix.pherf.rules.DataValue;
 import org.apache.phoenix.pherf.rules.RulesApplier;
+import org.apache.phoenix.pherf.util.PhoenixUtil;
 import org.joda.time.DateTime;
 import org.joda.time.format.DateTimeFormat;
 import org.joda.time.format.DateTimeFormatter;
@@ -32,7 +33,9 @@ import java.util.*;
 
 import static org.junit.Assert.*;
 
-public class RuleGeneratorTest extends BaseTestWithCluster {
+public class RuleGeneratorTest {
+    static PhoenixUtil util = new PhoenixUtil(true);
+    static final String matcherScenario = PherfConstants.SCENARIO_ROOT_PATTERN + ".xml";
 
     @Test
     public void testDateGenerator() throws Exception {
@@ -40,8 +43,6 @@ public class RuleGeneratorTest extends BaseTestWithCluster {
         DataModel model = parser.getDataModels().get(0);
         DataLoader loader = new DataLoader(parser);
         RulesApplier rulesApplier = loader.getRulesApplier();
-        int sampleSize = 100;
-        List<String> values = new ArrayList<>(sampleSize);
 
         for (Column dataMapping : model.getDataMappingColumns()) {
             if ((dataMapping.getType() == DataTypeMapping.DATE) && (dataMapping.getName().equals("CREATED_DATE"))) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a0dc609/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/SchemaReaderTest.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/SchemaReaderTest.java b/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/SchemaReaderTest.java
deleted file mode 100644
index e8d1321..0000000
--- a/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/SchemaReaderTest.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- *   or more contributor license agreements.  See the NOTICE file
- *   distributed with this work for additional information
- *   regarding copyright ownership.  The ASF licenses this file
- *   to you under the Apache License, Version 2.0 (the
- *   "License"); you may not use this file except in compliance
- *   with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- *   Unless required by applicable law or agreed to in writing, software
- *   distributed under the License is distributed on an "AS IS" BASIS,
- *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *   See the License for the specific language governing permissions and
- *   limitations under the License.
- */
-
-package org.apache.phoenix.pherf;
-
-import org.apache.phoenix.pherf.configuration.Column;
-import org.apache.phoenix.pherf.configuration.DataModel;
-import org.apache.phoenix.pherf.configuration.Scenario;
-import org.apache.phoenix.pherf.configuration.XMLConfigParser;
-import org.apache.phoenix.pherf.schema.SchemaReader;
-import org.junit.Test;
-
-import java.net.URL;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.sql.Connection;
-import java.util.ArrayList;
-import java.util.List;
-
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-public class SchemaReaderTest extends BaseTestWithCluster {
-
-	@Test
-    public void testSchemaReader() {
-        // Test for the unit test version of the schema files.
-        assertApplySchemaTest();
-    }
-
-    private void assertApplySchemaTest() {
-        try {
-            SchemaReader reader = new SchemaReader(".*datamodel/.*test.*sql");
-
-            List<Path> resources = new ArrayList<>(reader.getResourceList());
-            assertTrue("Could not pull list of schema files.", resources.size() > 0);
-            assertNotNull("Could not read schema file.", this.getClass().getResourceAsStream(
-                    PherfConstants.RESOURCE_DATAMODEL + "/" + resources.get(0).getFileName().toString()));
-            assertNotNull("Could not read schema file.", reader.resourceToString(resources.get(0)));
-            reader.applySchema();
-
-            Connection connection = null;
-            URL resourceUrl = getClass().getResource("/scenario/test_scenario.xml");
-            assertNotNull("Test data XML file is missing", resourceUrl);
-            connection = util.getConnection();
-            Path resourcePath = Paths.get(resourceUrl.toURI());
-            DataModel data = XMLConfigParser.readDataModel(resourcePath);
-            List<Scenario> scenarioList = data.getScenarios();
-            Scenario scenario = scenarioList.get(0);
-            List<Column> columnList = util.getColumnsFromPhoenix(scenario.getSchemaName(), scenario.getTableNameWithoutSchemaName(), connection);
-            assertTrue("Could not retrieve Metadata from Phoenix", columnList.size() > 0);
-        } catch (Exception e) {
-            fail("Could not initialize SchemaReader");
-            e.printStackTrace();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a0dc609/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/TestHBaseProps.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/TestHBaseProps.java b/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/TestHBaseProps.java
index 0e8b6d3..b2712c4 100644
--- a/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/TestHBaseProps.java
+++ b/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/TestHBaseProps.java
@@ -30,6 +30,5 @@ public class TestHBaseProps {
 	public void testCheckHBaseProps(){
 		Configuration conf = HBaseConfiguration.create();
 		assertTrue("did not get correct threadpool size", conf.get("phoenix.query.threadPoolSize").equals("128"));
-		assertTrue("did not get correct concurrentrequests size", conf.get("hbase.sfdc.concurrentrequests.max").equals("2147483647"));
 	}
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a0dc609/phoenix-pherf/src/test/resources/datamodel/test_schema.sql
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/test/resources/datamodel/test_schema.sql b/phoenix-pherf/src/test/resources/datamodel/test_schema.sql
index 8f16675..162d288 100644
--- a/phoenix-pherf/src/test/resources/datamodel/test_schema.sql
+++ b/phoenix-pherf/src/test/resources/datamodel/test_schema.sql
@@ -1,19 +1,20 @@
--- 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.
-
+/*
+  -- 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_TABLE (
     TENANT_ID CHAR(15) NOT NULL,
     PARENT_ID CHAR(15) NOT NULL,

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a0dc609/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 a13497f..45d36d2 100644
--- a/phoenix-pherf/src/test/resources/scenario/test_scenario.xml
+++ b/phoenix-pherf/src/test/resources/scenario/test_scenario.xml
@@ -17,7 +17,7 @@
   ~   limitations under the License.
   -->
 
-<datamodel release="192" name="test_scenario">
+<datamodel name="test_scenario">
     <datamapping>
         <column>
             <!-- This column type defines what will generally happen to VARCHAR fields unless they are explicitly defined or overridden elsewhere -->
@@ -130,7 +130,7 @@
 
     </datamapping>
     <scenarios>
-        <scenario tableName="PHERF.TEST_TABLE" rowCount="50" name="testScenario">
+        <scenario tableName="PHERF.TEST_TABLE" rowCount="10" name="testScenario">
             <!-- Scenario level rule overrides will be unsupported in V1.
                     You can use the general datamappings in the mean time-->
             <dataOverride>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a0dc609/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 54fabbe..6c756c3 100644
--- a/pom.xml
+++ b/pom.xml
@@ -28,9 +28,9 @@
     <module>phoenix-pig</module>
     <module>phoenix-server-client</module>
     <module>phoenix-server</module>
-    <module>phoenix-assembly</module>
     <module>phoenix-pherf</module>
     <module>phoenix-spark</module>
+    <module>phoenix-assembly</module>
   </modules>
 
   <repositories>