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

[5/6] phoenix git commit: PHOENIX-39. ADding Pherf as a Phoenix module. Changed assembly and pom to build Pherf and include *.zip into distribution.

PHOENIX-39. ADding Pherf as a Phoenix module. Changed assembly and pom to build Pherf and include *.zip into distribution.


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

Branch: refs/heads/4.0
Commit: 4b51655922408dbc2ae58f6ca332c5fb2caeb84d
Parents: 09c8dd7
Author: Mujtaba <mu...@apache.org>
Authored: Fri Mar 13 10:38:54 2015 -0700
Committer: Mujtaba <mu...@apache.org>
Committed: Fri Mar 13 10:38:54 2015 -0700

----------------------------------------------------------------------
 .../src/build/components/all-common-files.xml   |  11 +-
 phoenix-pherf/README.md                         | 105 ++++++
 phoenix-pherf/cluster/pherf.sh                  |  33 ++
 .../config/datamodel/user_defined_schema.sql    |  27 ++
 phoenix-pherf/config/env.sh                     |  32 ++
 phoenix-pherf/config/pherf.properties           |  31 ++
 .../config/scenario/user_defined_scenario.xml   | 134 +++++++
 phoenix-pherf/pom.xml                           | 290 ++++++++++++++
 phoenix-pherf/src/main/assembly/cluster.xml     |  52 +++
 phoenix-pherf/src/main/assembly/standalone.xml  |  52 +++
 .../java/org/apache/phoenix/pherf/Pherf.java    | 201 ++++++++++
 .../apache/phoenix/pherf/PherfConstants.java    |  63 ++++
 .../phoenix/pherf/configuration/Column.java     | 210 +++++++++++
 .../phoenix/pherf/configuration/DataModel.java  |  75 ++++
 .../pherf/configuration/DataOverride.java       |  36 ++
 .../pherf/configuration/DataSequence.java       |  23 ++
 .../pherf/configuration/DataTypeMapping.java    |  46 +++
 .../pherf/configuration/ExecutionType.java      |  23 ++
 .../phoenix/pherf/configuration/Query.java      | 136 +++++++
 .../phoenix/pherf/configuration/QuerySet.java   | 130 +++++++
 .../phoenix/pherf/configuration/Scenario.java   | 163 ++++++++
 .../pherf/configuration/XMLConfigParser.java    | 157 ++++++++
 .../pherf/exception/FileLoaderException.java    |  28 ++
 .../exception/FileLoaderRuntimeException.java   |  28 ++
 .../phoenix/pherf/exception/PherfException.java |  30 ++
 .../pherf/exception/PherfRuntimeException.java  |  30 ++
 .../phoenix/pherf/jmx/MonitorDetails.java       |  50 +++
 .../phoenix/pherf/jmx/MonitorManager.java       | 173 +++++++++
 .../java/org/apache/phoenix/pherf/jmx/Stat.java |  32 ++
 .../jmx/monitors/CPULoadAverageMonitor.java     |  33 ++
 .../pherf/jmx/monitors/ExampleMonitor.java      |  33 ++
 .../pherf/jmx/monitors/FreeMemoryMonitor.java   |  30 ++
 .../GarbageCollectorElapsedTimeMonitor.java     |  44 +++
 .../pherf/jmx/monitors/HeapMemoryMonitor.java   |  32 ++
 .../pherf/jmx/monitors/MaxMemoryMonitor.java    |  30 ++
 .../phoenix/pherf/jmx/monitors/Monitor.java     |  30 ++
 .../jmx/monitors/NonHeapMemoryMonitor.java      |  32 ++
 .../ObjectPendingFinalizationCountMonitor.java  |  33 ++
 .../pherf/jmx/monitors/ThreadMonitor.java       |  32 ++
 .../pherf/jmx/monitors/TotalMemoryMonitor.java  |  30 ++
 .../phoenix/pherf/loaddata/DataLoader.java      | 365 ++++++++++++++++++
 .../pherf/result/DataLoadThreadTime.java        |  95 +++++
 .../pherf/result/DataLoadTimeSummary.java       |  84 +++++
 .../phoenix/pherf/result/DataModelResult.java   |  77 ++++
 .../phoenix/pherf/result/QueryResult.java       | 141 +++++++
 .../phoenix/pherf/result/QuerySetResult.java    |  47 +++
 .../org/apache/phoenix/pherf/result/Result.java |  54 +++
 .../phoenix/pherf/result/ResultHandler.java     |  37 ++
 .../phoenix/pherf/result/ResultManager.java     | 103 +++++
 .../apache/phoenix/pherf/result/ResultUtil.java | 271 +++++++++++++
 .../phoenix/pherf/result/ResultValue.java       |  40 ++
 .../apache/phoenix/pherf/result/RunTime.java    | 114 ++++++
 .../phoenix/pherf/result/ScenarioResult.java    |  47 +++
 .../apache/phoenix/pherf/result/ThreadTime.java | 141 +++++++
 .../phoenix/pherf/result/file/Extension.java    |  38 ++
 .../phoenix/pherf/result/file/Header.java       |  41 ++
 .../pherf/result/file/ResultFileDetails.java    |  46 +++
 .../pherf/result/impl/CSVResultHandler.java     | 139 +++++++
 .../pherf/result/impl/ImageResultHandler.java   | 127 +++++++
 .../pherf/result/impl/XMLResultHandler.java     | 103 +++++
 .../apache/phoenix/pherf/rules/DataValue.java   |  89 +++++
 .../phoenix/pherf/rules/RulesApplier.java       | 377 +++++++++++++++++++
 .../phoenix/pherf/schema/SchemaReader.java      |  97 +++++
 .../apache/phoenix/pherf/util/PhoenixUtil.java  | 199 ++++++++++
 .../apache/phoenix/pherf/util/ResourceList.java | 214 +++++++++++
 .../phoenix/pherf/util/RowCalculator.java       |  78 ++++
 .../pherf/workload/MultithreadedDiffer.java     | 113 ++++++
 .../pherf/workload/MultithreadedRunner.java     | 170 +++++++++
 .../phoenix/pherf/workload/QueryExecutor.java   | 246 ++++++++++++
 .../phoenix/pherf/workload/QueryVerifier.java   | 195 ++++++++++
 .../pherf/workload/WorkloadExecutor.java        | 115 ++++++
 .../datamodel/create_prod_test_unsalted.sql     |  33 ++
 phoenix-pherf/src/main/resources/hbase-site.xml |  25 ++
 .../scenario/prod_test_unsalted_scenario.xml    | 342 +++++++++++++++++
 .../phoenix/pherf/BaseTestWithCluster.java      |  67 ++++
 .../org/apache/phoenix/pherf/ColumnTest.java    |  50 +++
 .../phoenix/pherf/ConfigurationParserTest.java  | 200 ++++++++++
 .../apache/phoenix/pherf/DataIngestTest.java    |  78 ++++
 .../apache/phoenix/pherf/DataLoaderTest.java    | 108 ++++++
 .../org/apache/phoenix/pherf/PherfTest.java     |  67 ++++
 .../org/apache/phoenix/pherf/ResourceTest.java  |  68 ++++
 .../org/apache/phoenix/pherf/ResultTest.java    | 209 ++++++++++
 .../apache/phoenix/pherf/RuleGeneratorTest.java | 213 +++++++++++
 .../apache/phoenix/pherf/SchemaReaderTest.java  |  73 ++++
 .../apache/phoenix/pherf/TestHBaseProps.java    |  35 ++
 .../test/resources/datamodel/test_schema.sql    |  15 +
 phoenix-pherf/src/test/resources/hbase-site.xml |  25 ++
 .../src/test/resources/pherf.test.properties    |  47 +++
 .../test/resources/scenario/test_scenario.xml   | 161 ++++++++
 phoenix-pherf/standalone/pherf.sh               |  28 ++
 pom.xml                                         |   1 +
 91 files changed, 8607 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/4b516559/phoenix-assembly/src/build/components/all-common-files.xml
----------------------------------------------------------------------
diff --git a/phoenix-assembly/src/build/components/all-common-files.xml b/phoenix-assembly/src/build/components/all-common-files.xml
index 60f9159..c9e4691 100644
--- a/phoenix-assembly/src/build/components/all-common-files.xml
+++ b/phoenix-assembly/src/build/components/all-common-files.xml
@@ -62,10 +62,19 @@
       <fileMode>0644</fileMode>
       <directoryMode>0755</directoryMode>
     </fileSet>
-      <fileSet>
+    <fileSet>
       <directory>${project.basedir}/../examples</directory>
       <fileMode>0644</fileMode>
       <directoryMode>0755</directoryMode>
     </fileSet>
+      <fileSet>
+          <directory>${project.basedir}/../phoenix-pherf/target</directory>
+          <outputDirectory>phoenix-pherf</outputDirectory>
+          <includes>
+              <include>*.zip</include>
+          </includes>
+          <fileMode>0644</fileMode>
+          <directoryMode>0755</directoryMode>
+      </fileSet>
   </fileSets>
 </component>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4b516559/phoenix-pherf/README.md
----------------------------------------------------------------------
diff --git a/phoenix-pherf/README.md b/phoenix-pherf/README.md
new file mode 100644
index 0000000..e3e4213
--- /dev/null
+++ b/phoenix-pherf/README.md
@@ -0,0 +1,105 @@
+Pherf is a performance test framework that exercises HBase through Apache Phoenix, a SQL layer interface.
+
+## Build 
+mvn clean package -DskipTests
+
+## Important arguments:
+
+- -h _Help_ <br />
+- -l _Apply schema and load data_<br/>
+- -q _Executes Multi-threaded query sets and write results_<br/>
+- -z [quorum] _Zookeeper quorum_</br>
+- -m _Enable monitor for statistics_<br/>
+- -monitorFrequency [frequency in Ms] _Frequency at which the monitor will snopshot stats to log file. <br/>
+- -drop _Regex drop all tables with schema name as PHERF. Example drop Event tables: -drop .*(EVENT).* Drop all: -drop .* or -drop all_<br/>
+- -scenarioFile _Regex or file name of a specific scenario file to run._ <br />
+- -schemaFile _Regex or file name of a specific schema file to run._ <br />
+- -export Exports query results to CSV files in CSV_EXPORT directory <br />
+- -diff Compares results with previously exported results <br />
+- -hint _Executes all queries with specified hint. Example SMALL_ <br />
+- -rowCountOverride
+- -rowCountOverride [number of rows] _Specify number of rows to be upserted rather than using row count specified in schema_ </br>
+
+## Running from IDE
+Ex. Load data and execute queries. Specify the following as your IDE debug arguments:<br/> 
+`-drop -l -q -z localhost`
+
+## Running from command line
+Ex. Drop existing tables, load data, and execute queries:<br/>  
+`java -jar pherf-1.0-SNAPSHOT-jar-with-dependencies.jar -drop -l -q -z localhost`
+
+## Adding Rules for Data Creation
+Review [test_scenario.xml](/src/test/resources/scenario/test_scenario.xml) 
+for syntax examples.<br />
+
+* Rules are defined as `<columns />` and are applied in the order they appear in file.
+* Rules of the same type override the values of a prior rule of the same type. If `<userDefined>true</userDefined>` is 
+set, rule will only
+apply override when type and name match the column name in Phoenix.
+* `<prefix>` tag is set at the column level. It can be used to define a constant string appended to the beginning of 
+CHAR and VARCHAR data type values. 
+* **Required field** Supported Phoenix types: VARCHAR, CHAR, DATE, DECIMAL, INTEGER
+    * denoted by the `<type>` tag
+* User defined true changes rule matching to use both name and type fields to determine equivalence.
+    * Default is false if not specified and equivalence will be determined by type only. **An important note here is that you can still override rules without the user defined flag, but they will change the rule globally and not just for a specified column.**
+* **Required field** Supported Data Sequences
+    * RANDOM:       Random value which can be bound by other fields such as length.
+    * SEQUENTIAL:   Monotonically increasing long prepended to random strings.
+        * Only supported on VARCHAR and CHAR types
+    * LIST:         Means pick values from predefined list of values
+* **Required field** Length defines boundary for random values for CHAR and VARCHAR types.
+    * denoted by the `<length>` tag
+* Column level Min/Max value defines boundaries for numerical values. For DATES, these values supply a range between 
+which values are generated. At the column level the granularity is a year. At a specific data value level, the 
+granularity is down to the Ms.
+    * denoted by the `<minValue>` tag
+    * denoted by the `<maxValue>` tag
+* Null chance denotes the probability of generating a null value. From \[0-100\]. The higher the number, the more likely
+the value will be null.
+    * denoted by `<nullChance>`
+* Name can either be any text or the actual column name in the Phoenix table.
+    * denoted by the `<name>`
+* Value List is used in conjunction with LIST data sequences. Each entry is a DataValue with a specified value to be 
+used when generating data. 
+    * Denoted by the `<valueList><datavalue><value/></datavalue></valueList>` tags
+    * If the distribution attribute on the datavalue is set, values will be created according to
+that probability. 
+    * When distribution is used, values must add up to 100%. 
+    * If distribution is not used, values will be randomly picked from the list with equal distribution.
+
+## Defining Scenario
+Scenario can have multiple querySets. Consider following example, concurrency of 1-4 means that each query will be 
+executed starting with concurrency level of 1 and reach up to maximum concurrency of 4. Per thread, query would be 
+executed to a minimum of 10 times or 10 seconds (whichever comes first). QuerySet by defult is executed serially but you
+ can change executionType to PARALLEL so queries are executed concurrently. Scenarios are defined in XMLs stored 
+ in the resource directory.
+
+```
+
+<scenarios>
+    <!--Minimum of executionDurationInMs or numberOfExecutions. Which ever is reached first -->
+    <querySet concurrency="1-4" executionType="PARALLEL" executionDurationInMs="10000" numberOfExecutions="10">
+        <query id="q1" verifyRowCount="false" statement="select count(*) from PHERF.TEST_TABLE"/>
+        <query id="q2" tenantId="1234567890" ddl="create view if not exists 
+        myview(mypk varchar not null primary key, mycol varchar)" statement="upsert select ..."/>
+    </querySet>
+    <querySet concurrency="3" executionType="SERIAL" executionDurationInMs="20000" numberOfExecutions="100">
+        <query id="q3" verifyRowCount="false" statement="select count(*) from PHERF.TEST_TABLE"/>
+        <query id="q4" statement="select count(*) from PHERF.TEST_TABLE WHERE TENANT_ID='00D000000000062'"/>
+    </querySet>
+</scenario>
+        
+```
+
+## Results
+Results are written real time in _results_ directory. Open the result that is saved in .jpg format for real time 
+visualization.
+
+## Testing
+Default quorum is localhost. If you want to override set the system variable.
+
+Run unit tests: `mvn test -DZK_QUORUM=localhost`<br />
+Run a specific method: `mvn -Dtest=ClassName#methodName test` <br />
+To test on a real cluster: `./pherf.sh -drop all -l -q -z localhost -schemaFile .*user_defined_schema.sql -scenarioFile .*user_defined_scenario.xml`
+
+More to come...

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4b516559/phoenix-pherf/cluster/pherf.sh
----------------------------------------------------------------------
diff --git a/phoenix-pherf/cluster/pherf.sh b/phoenix-pherf/cluster/pherf.sh
new file mode 100755
index 0000000..5b02fcb
--- /dev/null
+++ b/phoenix-pherf/cluster/pherf.sh
@@ -0,0 +1,33 @@
+#!/bin/bash
+
+#  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.
+
+source config/env.sh
+HBASE_CLASSPATH=`$HBASE_ROOT/hbase/hbase/bin/hbase classpath`
+
+
+PHERF_HOME=$(cd "`dirname $0`" && pwd)
+CLASSPATH=${HBASE_CLASSPATH}
+CLASSPATH=${PHERF_HOME}/config:${CLASSPATH}
+
+for f in $PHERF_HOME/lib/*.jar; do
+  CLASSPATH=${CLASSPATH}:$f;
+done
+
+CMD=time $JAVA_HOME/bin/java $REMOTE_DEBUG -Dapp.home=$PHERF_HOME $ENV_PROPS -Xms512m -Xmx3072m -cp $CLASSPATH org.apache.phoenix.pherf.Pherf "$@"
+
+eval $CMD
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4b516559/phoenix-pherf/config/datamodel/user_defined_schema.sql
----------------------------------------------------------------------
diff --git a/phoenix-pherf/config/datamodel/user_defined_schema.sql b/phoenix-pherf/config/datamodel/user_defined_schema.sql
new file mode 100644
index 0000000..55dd82d
--- /dev/null
+++ b/phoenix-pherf/config/datamodel/user_defined_schema.sql
@@ -0,0 +1,27 @@
+/*
+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.USER_DEFINED_TEST (
+    TENANT_ID CHAR(15) NOT NULL,
+    CREATED_DATE DATE NOT NULL,
+    VAL_STRING VARCHAR
+    CONSTRAINT PK PRIMARY KEY
+    (
+        TENANT_ID,
+        CREATED_DATE DESC
+    )
+) VERSIONS=1,MULTI_TENANT=true,SALT_BUCKETS=16

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4b516559/phoenix-pherf/config/env.sh
----------------------------------------------------------------------
diff --git a/phoenix-pherf/config/env.sh b/phoenix-pherf/config/env.sh
new file mode 100644
index 0000000..0979415
--- /dev/null
+++ b/phoenix-pherf/config/env.sh
@@ -0,0 +1,32 @@
+#!/bin/sh
+
+#  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.
+
+# Required variable to point to Java installation
+JAVA_HOME=
+
+# Absolute path the the unzipped root directory of the HBase installation
+# This is required if you build using the default or cluster profile
+# Cluster profile assumes you want to pick up dependencies from HBase classpath
+# Not required in standalone.
+HBASE_ROOT=
+
+# Add a space seperated list of -D environment args. "-Dkey1-val1 -Dkey2=val2"
+ENV_PROPS=""
+
+# Uncomment if you would like to remotely debug
+#REMOTE_DEBUG="-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=6666"

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4b516559/phoenix-pherf/config/pherf.properties
----------------------------------------------------------------------
diff --git a/phoenix-pherf/config/pherf.properties b/phoenix-pherf/config/pherf.properties
new file mode 100644
index 0000000..354707a
--- /dev/null
+++ b/phoenix-pherf/config/pherf.properties
@@ -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.
+
+# General purpose thread pool size for Pherf. It's used for things like monitor threads.This should remain small
+# to limit the amount of background tasks sucking up resources away from tests.
+pherf.default.threadpool=10
+
+# Interval in Ms that the JMX monitors will take a snapshot and dump to log
+pherf.default.monitorFrequency=30000
+
+# Default number of writers to use when loading data
+# 0   - Set the number of writers to use all available cores
+# 1-N - ANy integer value for the number of threads to use
+pherf.default.dataloader.threadpool=0
+
+# When upserting, this is the max # of rows that will be inserted in a single commit
+pherf.default.dataloader.batchsize=1000
+

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4b516559/phoenix-pherf/config/scenario/user_defined_scenario.xml
----------------------------------------------------------------------
diff --git a/phoenix-pherf/config/scenario/user_defined_scenario.xml b/phoenix-pherf/config/scenario/user_defined_scenario.xml
new file mode 100644
index 0000000..e54d76a
--- /dev/null
+++ b/phoenix-pherf/config/scenario/user_defined_scenario.xml
@@ -0,0 +1,134 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+  ~ 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.
+  -->
+
+<datamodel release="192" name="test_scenario">
+    <datamapping>
+        <column>
+            <!-- This column type defines what will generally happen to VARCHAR fields unless they are explicitly defined or overridden elsewhere -->
+            <type>VARCHAR</type>
+            <dataSequence>RANDOM</dataSequence>
+            <length>15</length>
+            <name>GENERAL_VARCHAR</name>
+        </column>
+        <column>
+            <type>CHAR</type>
+            <dataSequence>SEQUENTIAL</dataSequence>
+            <length>15</length>
+            <name>GENERAL_CHAR</name>
+        </column>
+        <column>
+            <type>DATE</type>
+            <!--SEQUENTIAL is unsupported for DATE -->
+            <dataSequence>RANDOM</dataSequence>
+            <!-- Number [0-100] that represents the probability of creating a null value -->
+            <!-- The higher the number, the more like the value will returned will be null -->
+            <!-- Leaving this tag out is equivalent to having a 0 probability. i.e. never null -->
+            <nullChance>0</nullChance>
+            <minValue>1975</minValue>
+            <maxValue>2025</maxValue>
+            <name>GENERAL_DATE</name>
+        </column>
+        <column>
+            <type>DECIMAL</type>
+            <dataSequence>RANDOM</dataSequence>
+            <minValue>0</minValue>
+            <maxValue>1</maxValue>
+
+            <!-- Precision is limited to 18 -->
+            <precision>18</precision>
+            <!-- Number [0-100] that represents the probability of creating a null value -->
+            <!-- The higher the number, the more like the value will returned will be null -->
+            <!-- Leaving this tag out is equivalent to having a 0 probability. i.e. never null -->
+            <nullChance>10</nullChance>
+            <name>GENERAL_DECIMAL</name>
+        </column>
+        <column>
+            <type>INTEGER</type>
+            <dataSequence>RANDOM</dataSequence>
+            <minValue>1</minValue>
+            <maxValue>50000000</maxValue>
+            <!-- Number [0-100] that represents the probability of creating a null value -->
+            <!-- The higher the number, the more like the value will returned will be null -->
+            <!-- Leaving this tag out is equivalent to having a 0 probability. i.e. never null -->
+            <nullChance>100</nullChance>
+            <name>GENERAL_INTEGER</name>
+        </column>
+        <column>
+            <type>DATE</type>
+            <name>CREATED_DATE</name>
+            <minValue>1975</minValue>
+            <maxValue>2025</maxValue>
+            <valuelist>
+                <!-- Distributes randomly with equal chance of being picked -->
+                <datavalue distribution="80">
+                    <!-- Joda time format: yyyy-MM-dd HH:mm:ss.SSS ZZZ -->
+                    <minValue>2019-09-15 00:01:00.000</minValue>
+                    <maxValue>2019-09-15 11:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="10">
+                    <value>2019-09-19 00:01:00</value>
+                </datavalue>
+                <datavalue distribution="10">
+                    <minValue>2019-09-22 00:01:00.000</minValue>
+                    <maxValue>2019-09-22 00:01:00.300</maxValue>
+                </datavalue>
+            </valuelist>
+        </column>
+        <column>
+            <type>CHAR</type>
+            <userDefined>true</userDefined>
+            <dataSequence>LIST</dataSequence>
+            <length>15</length>
+            <name>VAL_STRING</name>
+            <valuelist>
+                <!-- Distributes randomly with equal chance of being picked -->
+                <datavalue distribution="50">
+                    <value>KjhoOmnNbBs9kWs</value>
+                </datavalue>
+                <datavalue distribution="50">
+                    <value>VAL123</value>
+                </datavalue>
+            </valuelist>
+        </column>
+    </datamapping>
+    <scenarios>
+        <scenario tableName="PHERF.USER_DEFINED_TEST" rowCount="50" name="myscenario">
+            <!-- Scenario level rule overrides will be unsupported in V1.
+                    You can use the general datamappings in the mean time-->
+            <dataOverride>
+                <column>
+                    <type>VARCHAR</type>
+                    <userDefined>true</userDefined>
+                    <dataSequence>RANDOM</dataSequence>
+                    <length>10</length>
+                    <name>DO_NOT_USE</name>
+                </column>
+            </dataOverride>
+            <!--Note: 1. Minimum of executionDurationInMs or numberOfExecutions. Which ever is reached first 
+                      2. DDL included in query are executed only once on start of querySet execution.
+            -->
+            <querySet concurrency="1-3" executionType="SERIAL" executionDurationInMs="5000" numberOfExecutions="100">
+                <!-- queryGroup is a way to organize queries across tables or scenario files.
+                    The value will be dumped to results. This gives a value to group by on reporting to compare queries -->
+                <query id="q1" expectedAggregateRowCount="50"
+                       statement="select count(*) from PHERF.USER_DEFINED_TEST"/>
+            </querySet>
+        </scenario>
+    </scenarios>
+</datamodel>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4b516559/phoenix-pherf/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-pherf/pom.xml b/phoenix-pherf/pom.xml
new file mode 100644
index 0000000..dad3297
--- /dev/null
+++ b/phoenix-pherf/pom.xml
@@ -0,0 +1,290 @@
+<!--
+  ~ 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.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.phoenix</groupId>
+        <artifactId>phoenix</artifactId>
+        <version>4.4.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>pherf</artifactId>
+    <packaging>jar</packaging>
+    <!--<version>1.0-SNAPSHOT</version>-->
+    <name>Phoenix - Pherf</name>
+
+    <properties>
+    </properties>
+
+    <profiles>
+        <profile>
+            <id>standalone</id>
+            <activation>
+                <activeByDefault>true</activeByDefault>
+            </activation>
+            <build>
+                <resources>
+                    <resource>
+                        <directory>src/main/resources</directory>
+                    </resource>
+                </resources>
+                <testResources>
+                    <testResource>
+                        <directory>src/test/resources</directory>
+                    </testResource>
+                    <testResource>
+                        <directory>${project.basedir}/config</directory>
+                    </testResource>
+                </testResources>
+                <plugins>
+                    <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>
+                                <phase>package</phase>
+                                <goals>
+                                    <goal>single</goal>
+                                </goals>
+                                <configuration>
+                                    <descriptorRefs>
+                                        <descriptorRef>jar-with-dependencies</descriptorRef>
+                                    </descriptorRefs>
+                                    <archive>
+                                        <manifest>
+                                            <mainClass>org.apache.phoenix.pherf.Pherf</mainClass>
+                                        </manifest>
+                                    </archive>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>make-assembly</id>
+                                <phase>package</phase>
+                                <goals>
+                                    <goal>single</goal>
+                                </goals>
+                                <configuration>
+                                    <descriptors>
+                                        <descriptor>src/main/assembly/standalone.xml</descriptor>
+                                    </descriptors>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+        </profile>
+
+        <profile>
+            <!-- default: creates a bundle for running on an SFDC cluster node as the 'sfdc' user -->
+            <id>cluster</id>
+            <activation>
+                <activeByDefault>true</activeByDefault>
+            </activation>
+            <build>
+                <resources>
+                    <resource>
+                        <directory>src/main/resources</directory>
+                    </resource>
+                </resources>
+                <testResources>
+                    <testResource>
+                        <directory>src/test/resources</directory>
+                    </testResource>
+                    <testResource>
+                        <directory>${project.basedir}/config</directory>
+                    </testResource>
+                </testResources>
+                <plugins>
+                    <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>
+                                <phase>package</phase>
+                                <goals>
+                                    <goal>single</goal>
+                                </goals>
+                                <configuration>
+                                    <descriptorRefs>
+                                        <descriptorRef>jar-with-dependencies</descriptorRef>
+                                    </descriptorRefs>
+                                    <archive>
+                                        <manifest>
+                                            <mainClass>org.apache.phoenix.pherf.Pherf</mainClass>
+                                        </manifest>
+                                    </archive>
+                                </configuration>
+                            </execution>
+                            <execution>
+                                <id>make-assembly</id>
+                                <phase>package</phase>
+                                <goals>
+                                    <goal>single</goal>
+                                </goals>
+                                <configuration>
+                                    <descriptors>
+                                        <descriptor>src/main/assembly/cluster.xml</descriptor>
+                                    </descriptors>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+        </profile>
+
+    </profiles>
+    <repositories>
+        <repository>
+            <id>apache release</id>
+            <url>https://repository.apache.org/content/repositories/releases/</url>
+        </repository>
+    </repositories>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.phoenix</groupId>
+            <artifactId>phoenix-core</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.phoenix</groupId>
+            <artifactId>phoenix-core</artifactId>
+            <classifier>tests</classifier>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+            <version>14.0.1</version>
+        </dependency>
+        <dependency>
+            <groupId>commons-io</groupId>
+            <artifactId>commons-io</artifactId>
+            <version>2.4</version>
+        </dependency>
+        <dependency>
+            <groupId>commons-cli</groupId>
+            <artifactId>commons-cli</artifactId>
+            <version>1.2</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-lang3</artifactId>
+            <version>3.3.2</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-math3</artifactId>
+            <version>3.3</version>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <version>4.11</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.hamcrest</groupId>
+            <artifactId>hamcrest-junit</artifactId>
+            <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>
+        </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>
+    </dependencies>
+    <build>
+        <plugins>
+            <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>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
+</project>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4b516559/phoenix-pherf/src/main/assembly/cluster.xml
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/assembly/cluster.xml b/phoenix-pherf/src/main/assembly/cluster.xml
new file mode 100644
index 0000000..961240a
--- /dev/null
+++ b/phoenix-pherf/src/main/assembly/cluster.xml
@@ -0,0 +1,52 @@
+<!--
+  ~ 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.
+  -->
+
+<assembly xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.2"
+          xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+          xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.2 http://maven.apache.org/xsd/assembly-1.1.2.xsd">
+    <id>cluster</id>
+    <baseDirectory>/</baseDirectory>
+    <formats>
+        <format>zip</format>
+    </formats>
+    <fileSets>
+        <fileSet>
+            <directory>${project.basedir}/config</directory>
+            <outputDirectory>pherf-${project.version}-cluster/config</outputDirectory>
+            <includes>
+                <include>*/*</include>
+                <include>*</include>
+            </includes>
+        </fileSet>
+        <fileSet>
+            <directory>${project.basedir}/cluster</directory>
+            <outputDirectory>pherf-${project.version}-cluster/</outputDirectory>
+            <includes>
+                <include>*sh</include>
+                <include>*dependencies.jar</include>
+            </includes>
+        </fileSet>
+        <fileSet>
+            <directory>${project.build.directory}</directory>
+            <outputDirectory>pherf-${project.version}-cluster/lib</outputDirectory>
+            <includes>
+                <include>*dependencies.jar</include>
+            </includes>
+        </fileSet>
+    </fileSets>
+</assembly>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4b516559/phoenix-pherf/src/main/assembly/standalone.xml
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/assembly/standalone.xml b/phoenix-pherf/src/main/assembly/standalone.xml
new file mode 100644
index 0000000..7163427
--- /dev/null
+++ b/phoenix-pherf/src/main/assembly/standalone.xml
@@ -0,0 +1,52 @@
+<!--
+  ~ 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.
+  -->
+
+<assembly xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.2"
+          xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+          xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.2 http://maven.apache.org/xsd/assembly-1.1.2.xsd">
+    <id>standalone</id>
+    <baseDirectory>/</baseDirectory>
+    <formats>
+        <format>zip</format>
+    </formats>
+    <fileSets>
+        <fileSet>
+            <directory>${project.basedir}/config</directory>
+            <outputDirectory>pherf-${project.version}-standalone/config</outputDirectory>
+            <includes>
+                <include>*/*</include>
+                <include>*</include>
+            </includes>
+        </fileSet>
+        <fileSet>
+            <directory>${project.basedir}/standalone</directory>
+            <outputDirectory>pherf-${project.version}-standalone/</outputDirectory>
+            <includes>
+                <include>*sh</include>
+            </includes>
+        </fileSet>
+    </fileSets>
+    <dependencySets>
+        <dependencySet>
+            <unpack>false</unpack>
+            <outputDirectory>pherf-${project.version}-standalone/lib</outputDirectory>
+            <includes>
+            </includes>
+        </dependencySet>
+    </dependencySets>
+</assembly>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4b516559/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
new file mode 100644
index 0000000..ac386f2
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/Pherf.java
@@ -0,0 +1,201 @@
+/*
+ * 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.XMLConfigParser;
+import org.apache.phoenix.pherf.schema.SchemaReader;
+import org.apache.phoenix.pherf.util.PhoenixUtil;
+import org.apache.phoenix.pherf.util.ResourceList;
+import org.apache.phoenix.pherf.workload.WorkloadExecutor;
+
+import org.apache.commons.cli.*;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.file.Path;
+import java.util.Collection;
+import java.util.Properties;
+
+public class Pherf {
+    private static final Logger logger = LoggerFactory.getLogger(Pherf.class);
+    private static final Options options = new Options();
+
+    static {
+        options.addOption("m", "monitor", false, "Launch the stats profilers");
+        options.addOption("monitorFrequency", true, "Override for frequency in Ms for which monitor should log stats. " +
+                "\n See pherf.default.monitorFrequency in pherf.properties");
+        options.addOption("d", "debug", false, "Put tool in debug mode");
+        options.addOption("z", "zookeeper", true, "HBase Zookeeper address for connection. Default: localhost");
+        options.addOption("l", "load", false, "Loads data according to specified configuration values.");
+        options.addOption("scenarioFile", true, "Regex or file name for the Test Scenario configuration .xml file to use.");
+        options.addOption("drop", true, "Regex drop all tables with schema name as PHERF. " +
+                "\nExample drop Event tables: -drop .*(EVENT).* Drop all: -drop .* or -drop all");
+        options.addOption("schemaFile", true, "Regex or file name for the Test phoenix table schema .sql to use.");
+        options.addOption("rowCountOverride", true, "Row count override to use instead of one specified in scenario.");
+        options.addOption("hint", true, "Executes all queries with specified hint. Example SMALL");
+        options.addOption("diff", false, "Run pherf in verification mode and diff with exported results");
+        options.addOption("export", false, "Exports query results to CSV files in " + PherfConstants.EXPORT_DIR + " directory");
+        options.addOption("listFiles", false, "List available resource files");
+        options.addOption("writerThreadSize", true, "Override the default number of writer threads. " +
+                "See pherf.default.dataloader.threadpool in Pherf.properties.");
+        options.addOption("q", "query", false, "Executes multi-threaded query sets");
+        options.addOption("h", "help", false, "Get help on using this utility.");
+    }
+
+    private final String zookeeper;
+    private final String scenarioFile;
+    private final String schemaFile;
+    private final String queryHint;
+    private final Properties properties;
+    private final boolean loadData;
+    private final String dropPherfTablesRegEx;
+    private final boolean executeQuerySets;
+    private final boolean exportCSV;
+    private final boolean diff;
+    private final boolean monitor;
+    private final int rowCountOverride;
+    private  final boolean listFiles;
+
+    public Pherf(String[] args) throws Exception {
+        CommandLineParser parser = new PosixParser();
+        CommandLine command = null;
+        HelpFormatter hf = new HelpFormatter();
+
+        try {
+            command = parser.parse(options, args);
+        } catch (ParseException e) {
+            hf.printHelp("Pherf", options);
+            System.exit(1);
+        }
+
+        properties = getProperties();
+        dropPherfTablesRegEx = command.getOptionValue("drop", null);
+        monitor = command.hasOption("m");
+        String monitorFrequency = (command.hasOption("m") && command.hasOption("monitorFrequency"))
+                ? command.getOptionValue("monitorFrequency")
+                : properties.getProperty("pherf.default.monitorFrequency");
+        properties.setProperty("pherf.default.monitorFrequency", monitorFrequency);
+
+        logger.debug("Using Monitor: " + monitor);
+        logger.debug("Monitor Frequency Ms:" + monitorFrequency);
+        loadData = command.hasOption("l");
+        executeQuerySets = command.hasOption("q");
+        zookeeper = command.getOptionValue("z", "localhost");
+        queryHint = command.getOptionValue("hint", null);
+        exportCSV = command.hasOption("export");
+        diff = command.hasOption("diff");
+        listFiles = command.hasOption("listFiles");
+        scenarioFile = command.hasOption("scenarioFile") ? command.getOptionValue("scenarioFile") : null;
+        schemaFile = command.hasOption("schemaFile") ? command.getOptionValue("schemaFile") : null;
+        rowCountOverride = Integer.parseInt(command.getOptionValue("rowCountOverride", "0"));
+        String writerThreadPoolSize = command.getOptionValue("writerThreadSize",
+                properties.getProperty("pherf.default.dataloader.threadpool"));
+        properties.setProperty("pherf. default.dataloader.threadpool", writerThreadPoolSize);
+
+
+        if ((command.hasOption("h") || (args == null || args.length == 0))
+                && !command.hasOption("listFiles")) {
+            hf.printHelp("Pherf", options);
+            System.exit(1);
+        }
+        PhoenixUtil.setZookeeper(zookeeper);
+        PhoenixUtil.setRowCountOverride(rowCountOverride);
+        PhoenixUtil.writeSfdcClientProperty();
+    }
+
+    public static void main(String[] args) {
+        try {
+            new Pherf(args).run();
+        } catch (Exception e) {
+            e.printStackTrace();
+            System.exit(1);
+        }
+    }
+
+    public void run() throws Exception {
+        WorkloadExecutor workloadExec = null;
+        try {
+            if (listFiles) {
+                ResourceList list = new ResourceList(PherfConstants.RESOURCE_DATAMODEL);
+                Collection<Path> schemaFiles = list.getResourceList(PherfConstants.SCHEMA_ROOT_PATTERN + ".sql");
+                System.out.println("Schema Files:");
+                for (Path path : schemaFiles) {
+                    System.out.println(path);
+                }
+                list = new ResourceList(PherfConstants.RESOURCE_SCENARIO);
+                Collection<Path> scenarioFiles =
+                        list.getResourceList(PherfConstants.SCENARIO_ROOT_PATTERN + ".xml");
+                System.out.println("Scenario Files:");
+                for (Path path : scenarioFiles) {
+                    System.out.println(path);
+                }
+                return;
+            }
+            workloadExec = (scenarioFile == null)
+                    ? new WorkloadExecutor(properties,
+                    new XMLConfigParser(PherfConstants.DEFAULT_FILE_PATTERN),
+                    monitor)
+                    : new WorkloadExecutor(properties,
+                    new XMLConfigParser(scenarioFile),
+                    monitor);
+
+            // Drop tables with PHERF schema and regex comparison
+            if (null != dropPherfTablesRegEx) {
+                logger.info("\nDropping existing table with PHERF namename and "
+                        + dropPherfTablesRegEx + " regex expression.");
+                new PhoenixUtil().deleteTables(dropPherfTablesRegEx);
+            }
+
+            // Schema and Data Load
+            if (loadData) {
+                logger.info("\nStarting to apply schema...");
+                SchemaReader reader = (schemaFile == null)
+                        ? new SchemaReader(".*.sql")
+                        : new SchemaReader(schemaFile);
+                reader.applySchema();
+
+                logger.info("\nStarting Data Load...");
+                workloadExec.executeDataLoad();
+
+                logger.info("\nGenerate query gold files after data load");
+                workloadExec.executeMultithreadedQueryExecutor(queryHint, true, PherfConstants.RunMode.FUNCTIONAL);
+            } else {
+                logger.info("\nSKIPPED: Data Load and schema creation as -l argument not specified");
+            }
+
+            // Execute multi-threaded query sets
+            if (executeQuerySets) {
+                logger.info("\nStarting to apply schema...");
+                workloadExec.executeMultithreadedQueryExecutor(queryHint, exportCSV, diff ? PherfConstants.RunMode.FUNCTIONAL : PherfConstants.RunMode.PERFORMANCE);
+            } else {
+                logger.info("\nSKIPPED: Multithreaded query set execution as -q argument not specified");
+            }
+        } finally {
+            if (workloadExec != null) {
+                logger.info("Run completed. Shutting down Monitor if it was running.");
+                workloadExec.shutdown();
+            }
+        }
+    }
+
+    private static Properties getProperties() throws Exception {
+        ResourceList list = new ResourceList();
+        return list.getProperties();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4b516559/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
new file mode 100644
index 0000000..22d18f6
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/PherfConstants.java
@@ -0,0 +1,63 @@
+/*
+ * 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;
+
+public class PherfConstants {
+    public static final int DEFAULT_THREAD_POOL_SIZE = 10;
+    public static final int DEFAULT_BATCH_SIZE = 1000;
+    public static final String DEFAULT_DATE_PATTERN = "yyyy-MM-dd HH:mm:ss.SSS";
+    public static final String DEFAULT_FILE_PATTERN = ".*scenario.xml";
+    public static final String RESOURCE_SCENARIO = "/scenario";
+    public static final String
+            SCENARIO_ROOT_PATTERN =
+            ".*" + PherfConstants.RESOURCE_SCENARIO.substring(1) + ".*";
+    public static final String SCHEMA_ROOT_PATTERN = ".*";
+    public static final String PHERF_PROPERTIES = "pherf.properties";
+    public static final String RESULT_DIR = "RESULTS";
+    public static final String EXPORT_DIR = "CSV_EXPORT";
+    public static final String RESULT_PREFIX = "RESULT_";
+    public static final String PATH_SEPARATOR = "/";
+    public static final String RESULT_FILE_DELIMETER = ",";
+    public static final String NEW_LINE = "\n";
+
+    public static final long DEFAULT_NUMBER_OF_EXECUTIONS = 10;
+    public static final long DEFAULT_THREAD_DURATION_IN_MS = 10000;
+    public static final String DEFAULT_CONCURRENCY = "1";
+
+    public static final String DIFF_PASS = "VERIFIED_DIFF";
+    public static final String DIFF_FAIL = "FAILED_DIFF";
+
+    public static final String PHERF_SCHEMA_NAME = "PHERF";
+
+    // log out data load per n rows
+    public static final int LOG_PER_NROWS = 1000000;
+    public static final String COMBINED_FILE_NAME = "COMBINED";
+
+    public static final String EXPORT_TMP = EXPORT_DIR + "_TMP";
+    public static final String RESOURCE_DATAMODEL = "/datamodel";
+
+    // Default frequency in ms in which to log out monitor stats
+    public static final int MONITOR_FREQUENCY = 5000;
+    public static final String MONITOR_FILE_NAME = "STATS_MONITOR";
+
+    public static enum RunMode {
+        PERFORMANCE,
+        FUNCTIONAL
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4b516559/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/Column.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/Column.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/Column.java
new file mode 100644
index 0000000..a6b9d26
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/Column.java
@@ -0,0 +1,210 @@
+/*
+ * 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.configuration;
+
+import org.apache.phoenix.pherf.rules.DataValue;
+
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlElementWrapper;
+import java.util.List;
+
+public class Column {
+	private String name;
+    private String prefix;
+    private DataSequence dataSequence;
+    private int length, minValue, maxValue, precision;
+    private int nullChance;
+    private boolean userDefined;
+    private List<DataValue> dataValues;
+	private DataTypeMapping type;
+
+    public Column() {
+        super();
+        // Initialize int to negative value so we can distinguish 0 in mutations
+        // Object fields can be detected with null
+        this.length = Integer.MIN_VALUE;
+        this.minValue = Integer.MIN_VALUE;
+        this.maxValue = Integer.MIN_VALUE;
+        this.precision = Integer.MIN_VALUE;
+        this.nullChance = Integer.MIN_VALUE;
+        this.userDefined = false;
+    }
+
+    public Column(Column column) {
+        this();
+        this.type = column.type;
+        this.mutate(column);
+    }
+
+    /**
+     * Equal if column name and type match
+     * @param column
+     * @return
+     */
+    @Override
+    public boolean equals(Object column) {
+        Column col = (Column)column;
+        return (getType() == col.getType());
+    }
+
+    public String getName() {
+		return name;
+	}
+
+	public void setName(String name) {
+		this.name = name;
+	}
+
+	public DataSequence getDataSequence() {
+		return dataSequence;
+	}
+
+	public void setDataSequence(DataSequence dataSequence) {
+		this.dataSequence = dataSequence;
+	}
+
+	public int getLength() {
+		return length;
+	}
+
+	public void setLength(int length) {
+		this.length = length;
+	}
+
+	public DataTypeMapping getType() {
+		return type;
+	}
+
+	public void setType(DataTypeMapping type) {
+		this.type = type;
+	}
+
+    public int getMinValue() {
+        return minValue;
+    }
+
+    public void setMinValue(int minValue) {
+        this.minValue = minValue;
+    }
+
+    public int getMaxValue() {
+        return maxValue;
+    }
+
+    public void setMaxValue(int maxValue) {
+        this.maxValue = maxValue;
+    }
+
+    public int getPrecision() {
+        return precision;
+    }
+
+    public void setPrecision(int precision) {
+        this.precision = precision;
+    }
+
+    /**
+     * Changes fields of this object to match existing fields from the passed Column
+     * null object members are ignored.
+     *
+     * Field type cannot be mutated.
+     * @param column {@link Column}
+     *               obj contains only the fields you want to mutate this object into.
+     */
+    public void mutate(Column column) {
+        if (column.getMinValue() != Integer.MIN_VALUE) {
+            setMinValue(column.getMinValue());
+        }
+
+        if (column.getMaxValue() != Integer.MIN_VALUE) {
+            setMaxValue(column.getMaxValue());
+        }
+
+        if (column.getLength() != Integer.MIN_VALUE) {
+            setLength(column.getLength());
+        }
+
+        if (column.getName() != null) {
+            setName(column.getName());
+        }
+
+        if (column.getPrefix() != null) {
+            setPrefix(column.getPrefix());
+        }
+
+        if (column.getDataSequence() != null) {
+            setDataSequence(column.getDataSequence());
+        }
+
+        if (column.getNullChance() != Integer.MIN_VALUE) {
+            setNullChance(column.getNullChance());
+        }
+
+        if (column.getPrecision() != Integer.MIN_VALUE) {
+            setPrecision(column.getPrecision());
+        }
+
+        if (column.isUserDefined()) {
+            setUserDefined(column.isUserDefined());
+        }
+
+        if (column.dataValues != null) {
+           setDataValues(column.getDataValues());
+        }
+    }
+
+    public int getNullChance() {
+        return nullChance;
+    }
+
+    public void setNullChance(int nullChance) {
+        this.nullChance = nullChance;
+    }
+
+    public boolean isUserDefined() {
+        return userDefined;
+    }
+
+    public void setUserDefined(boolean userDefined) {
+        this.userDefined = userDefined;
+    }
+
+    public List<DataValue> getDataValues() {
+        return dataValues;
+    }
+
+    @XmlElementWrapper(name = "valuelist")
+    @XmlElement(name = "datavalue")
+    public void setDataValues(List<DataValue> dataValues) {
+        this.dataValues = dataValues;
+
+        // DataValue type is inherited from the column
+        for (DataValue value : dataValues) {
+            value.setType(getType());
+        }
+    }
+
+    public String getPrefix() {
+        return prefix;
+    }
+
+    public void setPrefix(String prefix) {
+        this.prefix = prefix;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4b516559/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/DataModel.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/DataModel.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/DataModel.java
new file mode 100644
index 0000000..d60fd9c
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/DataModel.java
@@ -0,0 +1,75 @@
+/*
+ * 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.configuration;
+
+import javax.xml.bind.annotation.XmlAttribute;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlElementWrapper;
+import javax.xml.bind.annotation.XmlRootElement;
+import java.util.List;
+
+@XmlRootElement(name = "datamodel")
+public class DataModel {
+    private String release;
+    private String name;
+    private List<Scenario> scenarios;
+    private List<Column> dataMappingColumns;
+
+    public DataModel() {
+    }
+
+    public String getRelease() {
+        return this.release;
+    }
+
+    @XmlAttribute()
+    public void setRelease(String release) {
+        this.release = release;
+    }
+
+    public List<Scenario> getScenarios() {
+        return scenarios;
+    }
+
+    @XmlElementWrapper(name = "datamapping")
+    @XmlElement(name = "column")
+    public void setDataMappingColumns(List<Column> dataMappingColumns) {
+        this.dataMappingColumns = dataMappingColumns;
+    }
+
+    public List<Column> getDataMappingColumns() {
+        return dataMappingColumns;
+    }
+
+    @XmlElementWrapper(name = "scenarios")
+    @XmlElement(name = "scenario")
+    public void setScenarios(List<Scenario> scenarios) {
+        this.scenarios = scenarios;
+    }
+
+	public String getName() {
+		return name;
+	}
+
+	@XmlAttribute()
+	public void setName(String name) {
+		this.name = name;
+	}
+}
+

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4b516559/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/DataOverride.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/DataOverride.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/DataOverride.java
new file mode 100644
index 0000000..91d2dc2
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/DataOverride.java
@@ -0,0 +1,36 @@
+/*
+ * 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.configuration;
+
+import java.util.List;
+
+import javax.xml.bind.annotation.XmlType;
+
+@XmlType
+public class DataOverride {
+	private List<Column> column;
+
+	public List<Column> getColumn() {
+		return column;
+	}
+
+	public void setColumn(List<Column> column) {
+		this.column = column;
+	}
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4b516559/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/DataSequence.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/DataSequence.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/DataSequence.java
new file mode 100644
index 0000000..056a913
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/DataSequence.java
@@ -0,0 +1,23 @@
+/*
+ * 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.configuration;
+
+public enum DataSequence {
+	RANDOM, SEQUENTIAL,LIST;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4b516559/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/DataTypeMapping.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/DataTypeMapping.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/DataTypeMapping.java
new file mode 100644
index 0000000..99d765c
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/DataTypeMapping.java
@@ -0,0 +1,46 @@
+/*
+ * 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.configuration;
+
+import java.sql.Types;
+
+public enum DataTypeMapping {
+    VARCHAR("VARCHAR", Types.VARCHAR),
+    CHAR("CHAR", Types.CHAR),
+    DECIMAL("DECIMAL", Types.DECIMAL),
+    INTEGER("INTEGER", Types.INTEGER),
+    DATE("DATE", Types.DATE);
+
+    private final String sType;
+    private final int dType;
+
+    private DataTypeMapping(String sType, int dType) {
+        this.dType = dType;
+        this.sType = sType;
+    }
+
+    @Override
+    public String toString() {
+        return this.sType;
+    }
+
+    public int getType() {
+        return this.dType;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4b516559/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/ExecutionType.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/ExecutionType.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/ExecutionType.java
new file mode 100644
index 0000000..998aa71
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/ExecutionType.java
@@ -0,0 +1,23 @@
+/*
+ * 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.configuration;
+
+public enum ExecutionType {
+	SERIAL, PARALLEL;
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4b516559/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/Query.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/Query.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/Query.java
new file mode 100644
index 0000000..1e5cabe
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/Query.java
@@ -0,0 +1,136 @@
+/*
+ * 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.configuration;
+
+import javax.xml.bind.annotation.XmlAttribute;
+import javax.xml.bind.annotation.XmlType;
+
+@XmlType
+public class Query {
+
+    private String statement;
+    private Long expectedAggregateRowCount;
+    private String tenantId;
+    private String ddl;
+    private String queryGroup;
+    private String id;
+
+    /**
+     * SQL statement
+     *
+     * @return
+     */
+    @XmlAttribute
+    public String getStatement() {
+        return statement;
+    }
+
+    public void setStatement(String statement) {
+        // normalize statement - merge all consecutive spaces into one
+        this.statement = statement.replaceAll("\\s+", " ");
+    }
+
+    /**
+     * Tenant Id used by connection of this query
+     *
+     * @return
+     */
+    @XmlAttribute
+    public String getTenantId() {
+        return tenantId;
+    }
+
+    public void setTenantId(String tenantId) {
+        this.tenantId = tenantId;
+    }
+
+    /**
+     * Expected aggregate row count is matched if specified
+     *
+     * @return
+     */
+    @XmlAttribute
+    public Long getExpectedAggregateRowCount() {
+        return expectedAggregateRowCount;
+    }
+
+    public void setExpectedAggregateRowCount(Long expectedAggregateRowCount) {
+        this.expectedAggregateRowCount = expectedAggregateRowCount;
+    }
+
+    /**
+     * DDL is executed only once. If tenantId is specified then DDL is executed with tenant
+     * specific connection.
+     *
+     * @return
+     */
+    @XmlAttribute
+    public String getDdl() {
+        return ddl;
+    }
+
+    public void setDdl(String ddl) {
+        this.ddl = ddl;
+    }
+
+    /**
+     * queryGroup attribute is just a string value to help correlate queries across sets or files.
+     * This helps to make sense of reporting results.
+     *
+     * @return the group id
+     */
+    @XmlAttribute
+    public String getQueryGroup() {
+        return queryGroup;
+    }
+
+    public void setQueryGroup(String queryGroup) {
+        this.queryGroup = queryGroup;
+    }
+
+    /**
+     * Set hint to query
+     *
+     * @param queryHint
+     */
+    public void setHint(String queryHint) {
+        if (null != queryHint) {
+            this.statement =
+                    this.statement.toUpperCase()
+                            .replace("SELECT ", "SELECT /*+ " + queryHint + "*/ ");
+        }
+    }
+
+    /**
+     * Query ID, Use UUID if none specified
+     *
+     * @return
+     */
+    @XmlAttribute
+    public String getId() {
+        if (null == this.id) {
+            this.id = java.util.UUID.randomUUID().toString();
+        }
+        return id;
+    }
+
+    public void setId(String id) {
+        this.id = id;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4b516559/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/QuerySet.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/QuerySet.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/QuerySet.java
new file mode 100644
index 0000000..3ab300c
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/QuerySet.java
@@ -0,0 +1,130 @@
+/*
+ * 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.configuration;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import javax.xml.bind.annotation.XmlAttribute;
+
+import org.apache.phoenix.pherf.PherfConstants;
+
+public class QuerySet {
+	private List<Query> query = new ArrayList<Query>();
+	private String concurrency = PherfConstants.DEFAULT_CONCURRENCY;
+	private long numberOfExecutions = PherfConstants.DEFAULT_NUMBER_OF_EXECUTIONS;
+	private long executionDurationInMs = PherfConstants.DEFAULT_THREAD_DURATION_IN_MS;
+	private ExecutionType executionType = ExecutionType.SERIAL;
+
+	/**
+	 * List of queries in each query set
+	 * @return
+	 */
+	public List<Query> getQuery() {
+		return query;
+	}
+
+	public void setQuery(List<Query> query) {
+		this.query = query;
+	}
+
+	/**
+	 * Target concurrency.
+	 * This can be set as a range. Example: 
+	 * 3
+	 * 1-4
+	 * @return
+	 */
+    @XmlAttribute
+	public String getConcurrency() {
+		return concurrency;
+	}
+
+	public void setConcurrency(String concurrency) {
+		this.concurrency = concurrency;
+	}
+	
+	/**
+	 * Number of execution of query per thread. Minimum of either number of executions
+	 * or execution duration is taken for each thread run
+	 * @return
+	 */
+	@XmlAttribute
+	public long getNumberOfExecutions() {
+		return numberOfExecutions;
+	}
+
+	public void setNumberOfExecutions(long numberOfExecutions) {
+		this.numberOfExecutions = numberOfExecutions;
+	}
+	
+	/**
+	 * Minimum concurrency level for a query set
+	 * @return
+	 */
+	public int getMinConcurrency() {
+		return getConcurrencyMinMax(0);
+	}
+	
+	/**
+	 * Maximum concurrency for a query set
+	 * @return
+	 */
+	public int getMaxConcurrency() {
+		return getConcurrencyMinMax(1);
+	}
+	
+	private int getConcurrencyMinMax(int idx) {
+		if (null == getConcurrency()) {
+			return 1;
+		}
+		String[] concurrencySplit = getConcurrency().split("-");
+		if (concurrencySplit.length == 2) {
+			return Integer.parseInt(concurrencySplit[idx]);
+		}
+		return Integer.parseInt(getConcurrency());
+	}
+
+	/**
+	 * This can be either SERIAL or PARALLEL
+	 * @return
+	 */
+	@XmlAttribute
+	public ExecutionType getExecutionType() {
+		return executionType;
+	}
+
+	public void setExecutionType(ExecutionType executionType) {
+		this.executionType = executionType;
+	}
+
+	/**
+	 * Execution duration of query per thread. Minimum of either number of executions
+	 * or execution duration is taken for each thread run
+	 * @return
+	 */
+	@XmlAttribute
+	public long getExecutionDurationInMs() {
+		return executionDurationInMs;
+	}
+
+	public void setExecutionDurationInMs(long executionDurationInMs) {
+		this.executionDurationInMs = executionDurationInMs;
+	}	
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4b516559/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
new file mode 100644
index 0000000..f75e528
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/Scenario.java
@@ -0,0 +1,163 @@
+/*
+ * 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.configuration;
+
+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 java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+@XmlRootElement(namespace = "org.apache.phoenix.pherf.configuration.DataModel")
+public class Scenario {
+    private String tableName;
+    private int rowCount;
+    private Map<String, String> phoenixProperties;
+    private DataOverride dataOverride;
+    private List<QuerySet> querySet = new ArrayList<QuerySet>();
+    private String name;
+
+    public Scenario() {
+    }
+
+    /**
+     * Scenarios have to have unique table names
+     *
+     * @param object
+     * @return
+     */
+    @Override
+    public boolean equals(Object object) {
+        Scenario scenario = (Scenario) object;
+        return (this.tableName.equals(scenario.getTableName()));
+    }
+
+    @Override
+    public int hashCode() {
+        return new HashCodeBuilder(11, 38).appendSuper(super.hashCode())
+                .append(tableName)
+                .toHashCode();
+    }
+
+    /**
+     * Table name for a scenario
+     *
+     * @return
+     */
+    @XmlAttribute()
+    public String getTableName() {
+        return tableName;
+    }
+
+    public void setTableName(String tableName) {
+        this.tableName = tableName;
+    }
+
+    /**
+     * Row count for a table
+     *
+     * @return
+     */
+    @XmlAttribute()
+    public int getRowCount() {
+        return PhoenixUtil.getRowCountOverride() == 0 ?
+                rowCount : PhoenixUtil.getRowCountOverride();
+    }
+
+    public void setRowCount(int rowCount) {
+        this.rowCount = rowCount;
+    }
+
+    /**
+     * Phoenix properties
+     *
+     * @return
+     */
+    public Map<String, String> getPhoenixProperties() {
+        return phoenixProperties;
+    }
+
+    public void setPhoenixProperties(Map<String, String> phoenixProperty) {
+        this.phoenixProperties = phoenixProperty;
+    }
+
+    /**
+     * Data override
+     *
+     * @return
+     */
+    @XmlElement()
+    public DataOverride getDataOverride() {
+        return dataOverride;
+    }
+
+    public void setDataOverride(DataOverride dataOverride) {
+        this.dataOverride = dataOverride;
+    }
+
+    /**
+     * List of Query Set
+     *
+     * @return
+     */
+    public List<QuerySet> getQuerySet() {
+        return querySet;
+    }
+
+    public void setQuerySet(List<QuerySet> querySet) {
+        this.querySet = querySet;
+    }
+
+    /**
+     * Extract schema name from table name
+     *
+     * @return
+     */
+    public String getSchemaName() {
+        return XMLConfigParser.parseSchemaName(this.tableName);
+    }
+
+    /**
+     * Extract table name without schema name
+     *
+     * @return
+     */
+    public String getTableNameWithoutSchemaName() {
+        return XMLConfigParser.parseTableName(this.tableName);
+    }
+
+    /**
+     * Name of scenario
+     *
+     * @return
+     */
+    @XmlAttribute()
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+}