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:45:30 UTC

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

Repository: phoenix
Updated Branches:
  refs/heads/master 94ade6ad4 -> 36b88651c


http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/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
new file mode 100644
index 0000000..8152390
--- /dev/null
+++ b/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ResultTest.java
@@ -0,0 +1,209 @@
+/*
+ * 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 static org.junit.Assert.*;
+
+import java.util.*;
+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;
+
+import org.apache.phoenix.pherf.result.file.Extension;
+import org.apache.phoenix.pherf.result.file.ResultFileDetails;
+import org.apache.phoenix.pherf.result.impl.CSVResultHandler;
+import org.apache.phoenix.pherf.result.impl.XMLResultHandler;
+import org.apache.phoenix.pherf.result.*;
+import org.junit.Test;
+
+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;
+        List<ResultValue> resultValues = new ArrayList<>();
+        for (String val : row) {
+            resultValues.add(new ResultValue(val));
+        }
+
+        try {
+            resultMonitorWriter = new CSVResultHandler(PherfConstants.MONITOR_FILE_NAME, ResultFileDetails.CSV_MONITOR);
+            Result
+                    result = new Result(ResultFileDetails.CSV_MONITOR, ResultFileDetails.CSV_MONITOR.getHeader().toString(), resultValues);
+            resultMonitorWriter.write(result);
+            resultMonitorWriter.write(result);
+            resultMonitorWriter.write(result);
+            resultMonitorWriter.flush();
+            List<Result> results = resultMonitorWriter.read();
+            assertEquals("Results did not contain row.", results.size(), 3);
+
+        } finally {
+            if (resultMonitorWriter != null) {
+                resultMonitorWriter.flush();
+                resultMonitorWriter.close();
+            }
+        }
+    }
+
+    @Test
+    public void testMonitorResult() throws Exception {
+        ExecutorService executorService = Executors.newFixedThreadPool(1);
+        MonitorManager monitor = new MonitorManager(100);
+        Future future = executorService.submit(monitor);
+        List<Result> records = null;
+        final int TIMEOUT = 30;
+
+        int ct = 0;
+        int max = 30;
+        // Wait while we write some rows.
+        while (!future.isDone()) {
+            Thread.sleep(100);
+            if (ct == max) {
+                int timer = 0;
+                monitor.stop();
+                while (monitor.isRunning() && (timer < TIMEOUT)) {
+                    System.out.println("Waiting for monitor to finish. Seconds Waited :" + timer);
+                    Thread.sleep(1000);
+                    timer++;
+                }
+            }
+
+            ct++;
+        }
+        executorService.shutdown();
+        records = monitor.readResults();
+
+        assertNotNull("Could not retrieve records", records);
+        assertEquals("Failed to get correct amount of CSV records.", records.size(), monitor.getRowCount());
+        assertFalse("Monitor was not stopped correctly.", monitor.isRunning());
+    }
+
+    @Test
+    public void testExtensionEnum() {
+        assertEquals("Extension did not match", Extension.CSV.toString(), ".csv");
+        assertEquals("Extension did not match", Extension.DETAILED_CSV.toString(), "_detail.csv");
+    }
+
+    @Test
+    public void testResult() throws Exception {
+        String filename = "testresult";
+        ResultHandler xmlResultHandler = null;
+        ResultManager resultManager = new ResultManager(filename, RunMode.PERFORMANCE);
+
+        // write result to file
+        DataModelResult dataModelResult = new DataModelResult();
+        dataModelResult.setZookeeper("mytestzk");
+        ScenarioResult scenarioResult = new ScenarioResult();
+        scenarioResult.setTableName("MY_TABLE_NAME");
+
+        // Scenario Name left blank on purpose to test that null values get generated correctly.
+        //scenarioResult.setName("MY_TEST_SCENARIO");
+
+        dataModelResult.getScenarioResult().add(scenarioResult);
+        scenarioResult.setRowCount(999);
+        QuerySetResult querySetResult = new QuerySetResult();
+        querySetResult.setConcurrency("50");
+        scenarioResult.getQuerySetResult().add(querySetResult);
+        Query query = new Query();
+        Query query2 = new Query();
+
+        // add some spaces so we test query gets normalized
+        query.setQueryGroup("g123");
+        query.setTenantId("tennantID123");
+        query.setStatement("Select    * \n" + "from    FHA");
+        query2.setStatement("Select a, b, c  * \n" + "from    FHA2");
+        assertEquals("Expected consecutive spaces to be normalized", "Select * from FHA", query.getStatement());
+
+        QueryResult queryResult = new QueryResult(query);
+        QueryResult queryResult2 = new QueryResult(query2);
+        querySetResult.getQueryResults().add(queryResult);
+        querySetResult.getQueryResults().add(queryResult2);
+
+        ThreadTime tt = new ThreadTime();
+        tt.setThreadName("thread1");
+        Calendar calendar = Calendar.getInstance();
+        Date startTime1 = calendar.getTime();
+        RunTime runtime1 = new RunTime(startTime1, 1000L, 10);
+        tt.getRunTimesInMs().add(runtime1);
+        calendar.add(Calendar.MINUTE, -1);
+        RunTime runtime2 = new RunTime(calendar.getTime(), 2000L, 20);
+        tt.getRunTimesInMs().add(runtime2);
+        calendar.add(Calendar.MINUTE, -1);
+        RunTime runtime3 = new RunTime(calendar.getTime(), 3000L, 30);
+        tt.getRunTimesInMs().add(runtime3);
+        queryResult.getThreadTimes().add(tt);
+        queryResult2.getThreadTimes().add(tt);
+
+        //resultUtil.writeResultToFile(dataModelResult, filename, RunMode.PERFORMANCE);
+        resultManager.write(dataModelResult);
+
+        // Put some stuff in a combined file
+        List<DataModelResult> modelResults = new ArrayList<>();
+        modelResults.add(dataModelResult);
+        modelResults.add(dataModelResult);
+        resultManager.write(modelResults);
+
+        // read result from file
+        xmlResultHandler = new XMLResultHandler(filename, ResultFileDetails.XML);
+        List<Result> resultList = xmlResultHandler.read();
+        ResultValue<DataModelResult> resultValue = resultList.get(0).getResultValues().get(0);
+        DataModelResult dataModelResultFromFile = resultValue.getResultValue();
+
+        ScenarioResult scenarioResultFromFile = dataModelResultFromFile.getScenarioResult().get(0);
+        QuerySetResult querySetResultFromFile = scenarioResultFromFile.getQuerySetResult().get(0);
+        QueryResult queryResultFromFile = querySetResultFromFile.getQueryResults().get(0);
+        ThreadTime ttFromFile = queryResultFromFile.getThreadTimes().get(0);
+
+        // thread level verification
+        assertEquals(10, (int) ttFromFile.getMinTimeInMs().getElapsedDurationInMs());
+        assertEquals(30, (int) ttFromFile.getMaxTimeInMs().getElapsedDurationInMs());
+        assertEquals(20, (int) ttFromFile.getAvgTimeInMs());
+        // 3rd runtime has the earliest start time, therefore that's what's expected.
+        assertEquals(runtime3.getStartTime(), ttFromFile.getStartTime());
+
+        assertEquals(runtime1.getResultRowCount(), ttFromFile.getRunTimesInMs().get(0).getResultRowCount());
+        assertEquals(runtime2.getResultRowCount(), ttFromFile.getRunTimesInMs().get(1).getResultRowCount());
+        assertEquals(runtime3.getResultRowCount(), ttFromFile.getRunTimesInMs().get(2).getResultRowCount());
+
+        // query result level verification
+        assertEquals(10, queryResultFromFile.getAvgMinRunTimeInMs());
+        assertEquals(30, queryResultFromFile.getAvgMaxRunTimeInMs());
+        assertEquals(20, queryResultFromFile.getAvgRunTimeInMs());
+        // 3rd runtime has the earliest start time, therefore that's what's expected.
+        assertEquals(runtime3.getStartTime(), queryResultFromFile.getStartTime());
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/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
new file mode 100644
index 0000000..8eafd56
--- /dev/null
+++ b/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/RuleGeneratorTest.java
@@ -0,0 +1,213 @@
+/*
+ * 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.*;
+import org.apache.phoenix.pherf.loaddata.DataLoader;
+import org.apache.phoenix.pherf.rules.DataValue;
+import org.apache.phoenix.pherf.rules.RulesApplier;
+import org.joda.time.DateTime;
+import org.joda.time.format.DateTimeFormat;
+import org.joda.time.format.DateTimeFormatter;
+import org.junit.Test;
+
+import java.sql.Types;
+import java.util.*;
+
+import static org.junit.Assert.*;
+
+public class RuleGeneratorTest extends BaseTestWithCluster {
+
+    @Test
+    public void testDateGenerator() throws Exception {
+        XMLConfigParser parser = new XMLConfigParser(matcherScenario);
+        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"))) {
+                // Test directly through generator method and that it converts to Phoenix type
+                assertRandomDateValue(dataMapping, rulesApplier);
+
+                // Test through data value method, which is normal path
+                // Do this 20 times and we should hit each possibility at least once.
+                for (int i = 0; i < 20; i++) {
+                    DataValue value = rulesApplier.getDataValue(dataMapping);
+                    assertNotNull("Could not retrieve DataValue for random DATE.", value);
+                    assertNotNull("Could not retrieve a value in DataValue for random DATE.", value.getValue());
+                    if (value.getMinValue() != null) {
+                        // Check that dates are between min/max
+                        assertDateBetween(value);
+                    }
+                }
+            }
+        }
+    }
+
+    @Test
+    public void testNullChance() throws Exception {
+        XMLConfigParser parser = new XMLConfigParser(matcherScenario);
+        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()) {
+            DataValue value = rulesApplier.getDataValue(dataMapping);
+            if (dataMapping.getNullChance() == 0) {
+                // 0 chance of getting null means we should never have an empty string returned
+                assertFalse("", value.getValue().equals(""));
+            } else if (dataMapping.getNullChance() == 100) {
+                // 100 chance of getting null means we should always have an empty string returned
+                assertTrue("", value.getValue().equals(""));
+            } else if ((dataMapping.getNullChance() == 90)) {
+                // You can't really test for this, but you can eyeball it on debugging.
+                for (int i = 0; i < sampleSize; i++) {
+                    DataValue tVal = rulesApplier.getDataValue(dataMapping);
+                    values.add(tVal.getValue());
+                }
+                Collections.sort(values);
+            }
+        }
+    }
+
+    @Test
+    public void testSequentialDataSequence() throws Exception {
+        XMLConfigParser parser = new XMLConfigParser(matcherScenario);
+        DataModel model = parser.getDataModels().get(0);
+        DataLoader loader = new DataLoader(parser);
+        RulesApplier rulesApplier = loader.getRulesApplier();
+
+        Column targetColumn = null;
+        for (Column column : model.getDataMappingColumns()) {
+            DataSequence sequence = column.getDataSequence();
+            if (sequence == DataSequence.SEQUENTIAL) {
+                targetColumn = column;
+                break;
+            }
+        }
+        assertNotNull("Could not find a DataSequence.SEQENTIAL rule.", targetColumn);
+        assertMultiThreadedIncrementValue(targetColumn, rulesApplier);
+    }
+
+    /**
+     * Verifies that we can generate a date between to specific dates.
+     *
+     * @param dataMapping
+     * @param rulesApplier
+     * @throws Exception
+     */
+    private void assertRandomDateValue(Column dataMapping, RulesApplier rulesApplier) throws Exception {
+        List<DataValue> dataValues = dataMapping.getDataValues();
+        DataValue ruleValue = dataValues.get(2);
+        String dt = rulesApplier.generateRandomDate(ruleValue.getMinValue(), ruleValue.getMaxValue());
+        ruleValue.setValue(dt);
+        assertDateBetween(ruleValue);
+    }
+
+    /**
+     * This method will test {@link org.apache.phoenix.pherf.configuration.DataSequence} SEQUENTIAL
+     * It ensures values returned always increase uniquely. RulesApplier will be accessed by multiple writer
+     * so we must ensure increment is thread safe.
+     */
+    private void assertMultiThreadedIncrementValue(final Column column, final RulesApplier rulesApplier) throws Exception {
+        final int threadCount = 30;
+        final int increments = 100;
+        final Set testSet = new TreeSet();
+        List<Thread> threadList = new ArrayList<>();
+        for (int i = 0; i < threadCount; i++) {
+            Thread t = new Thread() {
+
+                @Override
+                public void run() {
+                    for (int i = 0; i < increments; i++) {
+                        try {
+                            DataValue value = rulesApplier.getDataValue(column);
+                            String strValue = value.getValue();
+                            synchronized (testSet) {
+                                assertFalse("Incrementer gave a duplicate value: " + strValue, testSet.contains(strValue));
+                                assertTrue("Length did not equal expected.",
+                                        strValue.length() == column.getLength());
+                                testSet.add(strValue);
+                            }
+                        } catch (Exception e) {
+                            fail("Caught an exception during test: " + e.getMessage());
+                        }
+                    }
+                }
+            };
+            t.start();
+            threadList.add(t);
+        }
+
+        // Wait for threads to finish
+        for (Thread t : threadList) {
+            try {
+                t.join();
+            } catch (InterruptedException e) {
+                fail("There was a problem reading thread: " + e.getMessage());
+            }
+        }
+
+        assertTrue("Expected count in increments did not match expected", testSet.size() == (threadCount * increments));
+    }
+
+    @Test
+    public void testValueListRule() throws Exception {
+        List<String> expectedValues = new ArrayList();
+        expectedValues.add("aAAyYhnNbBs9kWk");
+        expectedValues.add("bBByYhnNbBs9kWu");
+        expectedValues.add("cCCyYhnNbBs9kWr");
+
+        XMLConfigParser parser = new XMLConfigParser(".*test_scenario.xml");
+        DataLoader loader = new DataLoader(parser);
+        RulesApplier rulesApplier = loader.getRulesApplier();
+        Scenario scenario = parser.getScenarios().get(0);
+
+        Column simPhxCol = new Column();
+        simPhxCol.setName("PARENT_ID");
+        simPhxCol.setType(DataTypeMapping.CHAR);
+
+        // Run this 10 times gives a reasonable chance that all the values will appear at least once
+        for (int i = 0; i < 10; i++) {
+            DataValue value = rulesApplier.getDataForRule(scenario, simPhxCol);
+            assertTrue("Got a value not in the list for the rule. :" + value.getValue(), expectedValues.contains(value.getValue()));
+        }
+    }
+
+    /**
+     * Asserts that the value field is between the min/max value fields
+     *
+     * @param value
+     */
+    private void assertDateBetween(DataValue value) {
+        DateTimeFormatter fmtr = DateTimeFormat.forPattern(PherfConstants.DEFAULT_DATE_PATTERN);
+
+        DateTime dt = fmtr.parseDateTime(value.getValue());
+        DateTime min = fmtr.parseDateTime(value.getMinValue());
+        DateTime max = fmtr.parseDateTime(value.getMaxValue());
+
+        assertTrue("Value " + dt + " is not after minValue", dt.isAfter(min));
+        assertTrue("Value " + dt + " is not before maxValue", dt.isBefore(max));
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/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
new file mode 100644
index 0000000..e8d1321
--- /dev/null
+++ b/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/SchemaReaderTest.java
@@ -0,0 +1,73 @@
+/*
+ * 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/36b88651/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
new file mode 100644
index 0000000..0e8b6d3
--- /dev/null
+++ b/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/TestHBaseProps.java
@@ -0,0 +1,35 @@
+/*
+ * 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 static org.junit.Assert.*;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.junit.Test;
+
+public class TestHBaseProps {
+	
+	@Test
+	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/36b88651/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
new file mode 100644
index 0000000..498f832
--- /dev/null
+++ b/phoenix-pherf/src/test/resources/datamodel/test_schema.sql
@@ -0,0 +1,15 @@
+CREATE TABLE IF NOT EXISTS PHERF.TEST_TABLE (
+    TENANT_ID CHAR(15) NOT NULL,
+    PARENT_ID CHAR(15) NOT NULL,
+    CREATED_DATE DATE NOT NULL,
+    FIELD VARCHAR,
+    OLDVAL_STRING VARCHAR,
+    NEWVAL_STRING VARCHAR,
+    SOME_INT INTEGER
+    CONSTRAINT PK PRIMARY KEY
+    (
+        TENANT_ID,
+        PARENT_ID,
+        CREATED_DATE DESC
+    )
+) VERSIONS=1,MULTI_TENANT=true,SALT_BUCKETS=16

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/test/resources/hbase-site.xml
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/test/resources/hbase-site.xml b/phoenix-pherf/src/test/resources/hbase-site.xml
new file mode 100644
index 0000000..4972828
--- /dev/null
+++ b/phoenix-pherf/src/test/resources/hbase-site.xml
@@ -0,0 +1,25 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+  ~ 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.
+  -->
+<configuration>
+    <property>
+        <name>phoenix.query.threadPoolSize</name>
+        <value>128</value>
+    </property>
+</configuration>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/test/resources/pherf.test.properties
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/test/resources/pherf.test.properties b/phoenix-pherf/src/test/resources/pherf.test.properties
new file mode 100644
index 0000000..9816fc8
--- /dev/null
+++ b/phoenix-pherf/src/test/resources/pherf.test.properties
@@ -0,0 +1,47 @@
+#
+# 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.
+#   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.
+#   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.
+#   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.
+#   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.
+#
+
+SOME_TEST_PROP=test_prop
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/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
new file mode 100644
index 0000000..a13497f
--- /dev/null
+++ b/phoenix-pherf/src/test/resources/scenario/test_scenario.xml
@@ -0,0 +1,161 @@
+<?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>PARENT_ID</name>
+            <valuelist>
+                <!-- Distributes according to specified values. These must total 100 -->
+                <datavalue distribution="60">
+                    <value>aAAyYhnNbBs9kWk</value>
+                </datavalue>
+                <datavalue distribution="20">
+                    <value>bBByYhnNbBs9kWu</value>
+                </datavalue>
+                <datavalue distribution="20">
+                    <value>cCCyYhnNbBs9kWr</value>
+                </datavalue>
+            </valuelist>
+        </column>
+        <column>
+            <!-- This column type defines what will generally happen to VARCHAR fields unless they are explicitly defined or overridden elsewhere -->
+            <type>VARCHAR</type>
+            <length>15</length>
+            <userDefined>true</userDefined>
+            <dataSequence>RANDOM</dataSequence>
+            <name>OLDVAL_STRING</name>
+            <prefix>MYPRFX</prefix>
+        </column>
+        <column>
+            <!-- This column type defines what will generally happen to VARCHAR fields unless they are explicitly defined or overridden elsewhere -->
+            <type>VARCHAR</type>
+            <length>15</length>
+            <userDefined>true</userDefined>
+            <dataSequence>SEQUENTIAL</dataSequence>
+            <name>NEWVAL_STRING</name>
+            <prefix>TSTPRFX</prefix>
+        </column>
+
+    </datamapping>
+    <scenarios>
+        <scenario tableName="PHERF.TEST_TABLE" rowCount="50" name="testScenario">
+            <!-- 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>FIELD</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">
+                <query id="q1" tenantId="123456789012345" expectedAggregateRowCount="0" statement="select count(*) from PHERF.TEST_TABLE"/>
+                <!-- 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="q2" queryGroup="g1" statement="select sum(SOME_INT) from PHERF.TEST_TABLE"/>
+            </querySet>
+            <!--Minimum of executionDurationInMs or numberOfExecutions. Which ever is reached first -->
+            <querySet concurrency="2-3" executionType="PARALLEL" executionDurationInMs="10000" numberOfExecutions="10">
+                <query id="q3" statement="select count(*) from PHERF.TEST_TABLE"/>
+                <query id="q4" statement="select sum(DIVISION) from PHERF.TEST_TABLE"/>
+            </querySet>
+        </scenario>
+    </scenarios>
+</datamodel>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/standalone/pherf.sh
----------------------------------------------------------------------
diff --git a/phoenix-pherf/standalone/pherf.sh b/phoenix-pherf/standalone/pherf.sh
new file mode 100755
index 0000000..483830c
--- /dev/null
+++ b/phoenix-pherf/standalone/pherf.sh
@@ -0,0 +1,28 @@
+#!/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
+PHERF_HOME=$(cd "`dirname $0`" && pwd)
+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/36b88651/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 74a8a36..c8d01b1 100644
--- a/pom.xml
+++ b/pom.xml
@@ -27,6 +27,7 @@
     <module>phoenix-flume</module>
     <module>phoenix-pig</module>
     <module>phoenix-assembly</module>
+    <module>phoenix-pherf</module>
   </modules>
 
   <repositories>


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

Posted by mu...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/util/RowCalculator.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/util/RowCalculator.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/util/RowCalculator.java
new file mode 100644
index 0000000..d61297a
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/util/RowCalculator.java
@@ -0,0 +1,78 @@
+/*
+ * 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.util;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class RowCalculator {
+    private final int buckets;
+    private final int rows;
+    private final List<Integer> rowCountList;
+
+    public RowCalculator(int buckets, int rows) {
+        this.buckets = buckets;
+        this.rows = rows;
+        this.rowCountList = Collections.synchronizedList(new ArrayList<Integer>(buckets));
+        init();
+    }
+
+    public synchronized int size() {
+        return rowCountList.size();
+    }
+
+    public synchronized int getNext() {
+        return rowCountList.remove(0);
+    }
+
+    /**
+     * Get the number of row that should fit into each bucket.
+     * @return
+     */
+    public int getRowsPerBucket() {
+        return rows / buckets;
+    }
+
+    /**
+     * Get the number of extra rows that need to be added if rows don't divide evenly among the buckets.
+     * @return
+     */
+    public int getExtraRowCount() {
+        return rows % buckets;
+    }
+
+    private void init() {
+        for (int i = 0; i < buckets; i++) {
+            synchronized (rowCountList) {
+                // On the first row count we tack on the extra rows if they exist
+                if (i == 0) {
+                    // When row count is small we just put them all in the first bucket
+                    if (rows < buckets) {
+                        rowCountList.add(getExtraRowCount());
+                    } else {
+                        rowCountList.add(getRowsPerBucket() + getExtraRowCount());
+                    }
+                } else {
+                    rowCountList.add(getRowsPerBucket());
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/MultithreadedDiffer.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/MultithreadedDiffer.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/MultithreadedDiffer.java
new file mode 100644
index 0000000..c78db90
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/MultithreadedDiffer.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *   or more contributor license agreements.  See the NOTICE file
+ *   distributed with this work for additional information
+ *   regarding copyright ownership.  The ASF licenses this file
+ *   to you under the Apache License, Version 2.0 (the
+ *   "License"); you may not use this file except in compliance
+ *   with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ */
+
+package org.apache.phoenix.pherf.workload;
+
+import java.util.Calendar;
+import java.util.Date;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.phoenix.pherf.PherfConstants;
+import org.apache.phoenix.pherf.configuration.Query;
+import org.apache.phoenix.pherf.result.RunTime;
+import org.apache.phoenix.pherf.result.ThreadTime;
+
+class MultithreadedDiffer implements Runnable {
+	private static final Logger logger = LoggerFactory
+			.getLogger(MultithreadedRunner.class);
+	private Thread t;
+	private Query query;
+	private ThreadTime threadTime;
+	private String threadName;
+	private long numberOfExecutions;
+	private long executionDurationInMs;
+	private QueryVerifier queryVerifier = new QueryVerifier(true);
+
+	private synchronized ThreadTime getThreadTime() {
+        return threadTime;
+    }
+
+    /**
+	 * Query Verification
+	 * @throws Exception
+	 */
+	private void diffQuery() throws Exception {
+		Long start = System.currentTimeMillis();
+		Date startDate = Calendar.getInstance().getTime();
+ 		String newCSV = queryVerifier.exportCSV(query);
+ 		boolean verifyResult = queryVerifier.doDiff(query, newCSV);
+ 		String explainPlan = queryVerifier.getExplainPlan(query);
+        getThreadTime().getRunTimesInMs().add(
+                new RunTime(verifyResult == true ? PherfConstants.DIFF_PASS : PherfConstants.DIFF_FAIL, 
+                		explainPlan, startDate, -1L, 
+                		(int)(System.currentTimeMillis() - start)));
+	}
+
+	/**
+	 * Multithreaded Differ
+	 * @param threadName
+	 * @param query
+	 * @param threadName
+	 * @param threadTime
+	 * @param numberOfExecutions
+	 * @param executionDurationInMs
+	 */
+	MultithreadedDiffer(String threadName,
+			Query query, 
+			ThreadTime threadTime, 
+			long numberOfExecutions, 
+			long executionDurationInMs) {
+		this.query = query;
+		this.threadName = threadName;
+		this.threadTime = threadTime;
+		this.numberOfExecutions = numberOfExecutions;
+		this.executionDurationInMs = executionDurationInMs;
+	}
+
+	/**
+	 * Executes verification runs for a minimum of number of execution or execution duration
+	 */
+	public void run() {
+		logger.info("\n\nThread Starting " + t.getName() + " ; " + query.getStatement() + " for "
+				+ numberOfExecutions + "times\n\n");
+		Long start = System.currentTimeMillis();
+		for (long i = numberOfExecutions; (i > 0 && ((System
+				.currentTimeMillis() - start) < executionDurationInMs)); i--) {
+			try {
+				diffQuery();
+			} catch (Exception e) {
+				e.printStackTrace();
+			}
+		}
+		logger.info("\n\nThread exiting." + t.getName() + "\n\n");
+	}
+
+	/**
+	 * Thread start
+	 * @return
+	 */
+	public Thread start() {
+		if (t == null) {
+			t = new Thread(this, threadName);
+			t.start();
+		}
+		return t;
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/MultithreadedRunner.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/MultithreadedRunner.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/MultithreadedRunner.java
new file mode 100644
index 0000000..237fc17
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/MultithreadedRunner.java
@@ -0,0 +1,170 @@
+/*
+ * 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.workload;
+
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.util.Calendar;
+import java.util.Date;
+
+import org.apache.phoenix.pherf.PherfConstants.RunMode;
+
+import org.apache.phoenix.pherf.result.DataModelResult;
+import org.apache.phoenix.pherf.result.ResultManager;
+import org.apache.phoenix.pherf.result.RunTime;
+import org.apache.phoenix.pherf.result.ThreadTime;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.phoenix.pherf.configuration.Query;
+import org.apache.phoenix.pherf.util.PhoenixUtil;
+
+class MultithreadedRunner implements Runnable {
+	private static final Logger logger = LoggerFactory
+			.getLogger(MultithreadedRunner.class);
+	private Thread t;
+	private Query query;
+	private ThreadTime threadTime;
+	private PhoenixUtil pUtil = new PhoenixUtil();
+	private String threadName;
+	private DataModelResult dataModelResult;
+	private long numberOfExecutions;
+	private long executionDurationInMs;
+	private static long lastResultWritten = System.currentTimeMillis() - 1000;
+    private final ResultManager resultManager;
+
+    /**
+     * Multithreaded runner
+     *
+     * @param threadName
+     * @param query
+     * @param dataModelResult
+     * @param threadTime
+     * @param numberOfExecutions
+     * @param executionDurationInMs
+     */
+    MultithreadedRunner(String threadName,
+                        Query query,
+                        DataModelResult dataModelResult,
+                        ThreadTime threadTime,
+                        long numberOfExecutions,
+                        long executionDurationInMs) {
+        this.query = query;
+        this.threadName = threadName;
+        this.threadTime = threadTime;
+        this.dataModelResult = dataModelResult;
+        this.numberOfExecutions = numberOfExecutions;
+        this.executionDurationInMs = executionDurationInMs;
+        this.resultManager = new ResultManager(dataModelResult.getName(), RunMode.PERFORMANCE);
+    }
+
+	/**
+	 * Executes run for a minimum of number of execution or execution duration
+	 */
+	public void run() {
+		logger.info("\n\nThread Starting " + t.getName() + " ; " + query.getStatement() + " for "
+				+ numberOfExecutions + "times\n\n");
+		Long start = System.currentTimeMillis();
+		for (long i = numberOfExecutions; (i > 0 && ((System
+				.currentTimeMillis() - start) < executionDurationInMs)); i--) {
+			try {
+                synchronized (resultManager) {
+                    timedQuery();
+                    if ((System.currentTimeMillis() - lastResultWritten) > 1000) {
+                        resultManager.write(dataModelResult);
+                        lastResultWritten = System.currentTimeMillis();
+                    }
+                }
+            } catch (Exception e) {
+				e.printStackTrace();
+			}
+		}
+		logger.info("\n\nThread exiting." + t.getName() + "\n\n");
+	}
+
+	/**
+	 * Thread start
+	 * @return
+	 */
+	public Thread start() {
+		if (t == null) {
+			t = new Thread(this, threadName);
+			t.start();
+		}
+		return t;
+	}
+
+    private synchronized ThreadTime getThreadTime() {
+        return threadTime;
+    }
+
+    /**
+     * Timed query execution
+     *
+     * @throws Exception
+     */
+    private void timedQuery() throws Exception {
+        boolean isSelectCountStatement = query.getStatement().toUpperCase().trim()
+                .contains("COUNT(*)") ? true : false;
+
+        Connection conn = null;
+        PreparedStatement statement = null;
+        ResultSet rs = null;
+        Long start = System.currentTimeMillis();
+        Date startDate = Calendar.getInstance().getTime();
+        String exception = null;
+        long resultRowCount = 0;
+
+        try {
+            conn = pUtil.getConnection(query.getTenantId());
+            statement = conn.prepareStatement(query.getStatement());
+            boolean isQuery = statement.execute();
+            if (isQuery) {
+                rs = statement.getResultSet();
+                while (rs.next()) {
+                    if (null != query.getExpectedAggregateRowCount()) {
+                        if (rs.getLong(1) != query.getExpectedAggregateRowCount())
+                            throw new RuntimeException("Aggregate count "
+                                    + rs.getLong(1) + " does not match expected "
+                                    + query.getExpectedAggregateRowCount());
+                    }
+
+                    if (isSelectCountStatement) {
+                        resultRowCount = rs.getLong(1);
+                    } else {
+                        resultRowCount++;
+                    }
+                }
+            } else {
+                conn.commit();
+            }
+        } catch (Exception e) {
+            e.printStackTrace();
+            exception = e.getMessage();
+        } finally {
+            getThreadTime().getRunTimesInMs().add(
+                    new RunTime(exception, startDate, resultRowCount, (int) (System.currentTimeMillis() - start)));
+
+            if (rs != null) rs.close();
+            if (statement != null) statement.close();
+            if (conn != null) conn.close();
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/QueryExecutor.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/QueryExecutor.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/QueryExecutor.java
new file mode 100644
index 0000000..5be5fc0
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/QueryExecutor.java
@@ -0,0 +1,246 @@
+/*
+ * 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.workload;
+
+import java.sql.Connection;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.phoenix.pherf.PherfConstants.RunMode;
+import org.apache.phoenix.pherf.configuration.XMLConfigParser;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.phoenix.pherf.result.*;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.phoenix.pherf.configuration.DataModel;
+import org.apache.phoenix.pherf.configuration.ExecutionType;
+import org.apache.phoenix.pherf.configuration.Query;
+import org.apache.phoenix.pherf.configuration.QuerySet;
+import org.apache.phoenix.pherf.configuration.Scenario;
+import org.apache.phoenix.pherf.util.PhoenixUtil;
+
+public class QueryExecutor {
+	private static final Logger logger = LoggerFactory.getLogger(QueryExecutor.class);
+	private List<DataModel> dataModels;
+	private String queryHint;
+	private RunMode runMode;
+    private final ResultUtil resultUtil;
+
+	public QueryExecutor(XMLConfigParser parser) {
+		this.dataModels = parser.getDataModels();
+        this.resultUtil = new ResultUtil();
+    }
+	
+	/**
+	 * Calls in Multithreaded Query Executor for all datamodels
+	 * @throws Exception 
+	 */
+	public void execute(String queryHint, boolean exportCSV, RunMode runMode) throws Exception {
+		this.queryHint = queryHint;
+		this.runMode = runMode;
+		for (DataModel dataModel: dataModels) {
+			if (exportCSV) {
+				exportAllScenarios(dataModel);	
+			} else {
+				executeAllScenarios(dataModel);
+			}
+		}
+	}
+
+	/**
+	 * Export all queries results to CSV 
+	 * @param dataModel
+	 * @throws Exception 
+	 */
+	protected void exportAllScenarios(DataModel dataModel) throws Exception {
+		List<Scenario> scenarios = dataModel.getScenarios();
+		QueryVerifier exportRunner = new QueryVerifier(false);
+		for (Scenario scenario : scenarios) {
+			for (QuerySet querySet : scenario.getQuerySet()) {
+				executeQuerySetDdls(querySet);
+				for (Query query : querySet.getQuery()) {
+					exportRunner.exportCSV(query);
+				}
+			}
+		}
+	}
+	
+	/**
+	 * Execute all scenarios
+	 * @param dataModel
+	 * @throws Exception 
+	 */
+	protected void executeAllScenarios(DataModel dataModel) throws Exception {
+		List<DataModelResult> dataModelResults = new ArrayList<DataModelResult>();
+		DataModelResult dataModelResult = new DataModelResult(dataModel, PhoenixUtil.getZookeeper());
+        ResultManager resultManager = new ResultManager(dataModelResult.getName(), this.runMode);
+
+
+		dataModelResults.add(dataModelResult);
+		List<Scenario> scenarios = dataModel.getScenarios();
+		Configuration conf = HBaseConfiguration.create();
+		Map<String, String> phoenixProperty = conf.getValByRegex("phoenix");
+		phoenixProperty.putAll(conf.getValByRegex("sfdc"));
+
+		for (Scenario scenario : scenarios) {
+			ScenarioResult scenarioResult = new ScenarioResult(scenario);
+			scenarioResult.setPhoenixProperties(phoenixProperty);
+			dataModelResult.getScenarioResult().add(scenarioResult);
+
+			for (QuerySet querySet : scenario.getQuerySet()) {
+				QuerySetResult querySetResult = new QuerySetResult(querySet);
+				scenarioResult.getQuerySetResult().add(querySetResult);
+				
+				executeQuerySetDdls(querySet);
+				
+				if (querySet.getExecutionType() == ExecutionType.SERIAL) {
+					execcuteQuerySetSerial(dataModelResult, querySet, querySetResult, scenarioResult);
+				} else {
+					execcuteQuerySetParallel(dataModelResult, querySet, querySetResult, scenarioResult);					
+				}
+			}
+            resultManager.write(dataModelResult);
+		}
+        resultManager.write(dataModelResults);
+	}
+
+	/**
+	 * Execute all querySet DDLs first based on tenantId if specified. This is executed
+	 * first since we don't want to run DDLs in parallel to executing queries.
+	 * 
+	 * @param querySet
+	 * @throws Exception 
+	 */
+	protected void executeQuerySetDdls(QuerySet querySet) throws Exception {
+		PhoenixUtil pUtil = new PhoenixUtil();
+		for (Query query : querySet.getQuery()) {
+			if (null != query.getDdl()) {
+				Connection conn = null;
+				try {
+					logger.info("\nExecuting DDL:" + query.getDdl() + " on tenantId:" + query.getTenantId());
+					pUtil.executeStatement(query.getDdl(), conn = pUtil.getConnection(query.getTenantId()));
+				} finally {
+					if (null != conn) {
+						conn.close();
+					}
+				}
+			}
+		}
+	}
+
+	/**
+	 * Execute query set serially
+	 * @param dataModelResult
+	 * @param querySet
+	 * @param querySetResult
+	 * @param scenario
+	 * @throws InterruptedException
+	 */
+	protected void execcuteQuerySetSerial(DataModelResult dataModelResult, QuerySet querySet, QuerySetResult querySetResult, Scenario scenario) throws InterruptedException {
+		for (Query query : querySet.getQuery()) {
+			QueryResult queryResult = new QueryResult(query);
+			querySetResult.getQueryResults().add(queryResult);
+
+			for (int cr = querySet.getMinConcurrency(); cr <= querySet
+					.getMaxConcurrency(); cr++) {
+				
+				List<Thread> threads = new ArrayList<Thread>();
+				
+				for (int i = 0; i < cr; i++) {
+
+					Thread thread = executeRunner((i + 1) + ","
+							+ cr, dataModelResult, queryResult,
+							querySetResult);
+					threads.add(thread);
+				}
+
+				for (Thread thread : threads) {
+					thread.join();
+				}
+			}
+		}
+	}
+
+	/**
+	 * Execute query set in parallel
+	 * @param dataModelResult
+	 * @param querySet
+	 * @param querySetResult
+	 * @param scenario
+	 * @throws InterruptedException
+	 */
+	protected void execcuteQuerySetParallel(DataModelResult dataModelResult, QuerySet querySet, QuerySetResult querySetResult, Scenario scenario)
+			throws InterruptedException {
+		for (int cr = querySet.getMinConcurrency(); cr <= querySet
+				.getMaxConcurrency(); cr++) {
+			List<Thread> threads = new ArrayList<Thread>();
+			for (int i = 0; i < cr; i++) {
+				for (Query query : querySet.getQuery()) {
+					QueryResult queryResult = new QueryResult(query);
+					querySetResult.getQueryResults().add(queryResult);
+
+					Thread thread = executeRunner((i + 1) + ","
+							+ cr, dataModelResult, queryResult,
+							querySetResult);
+					threads.add(thread);
+				}
+			}
+			for (Thread thread : threads) {
+				thread.join();
+			}
+		}
+	}
+	
+	/**
+	 * Execute multi-thread runner
+	 * @param name
+	 * @param dataModelResult
+	 * @param queryResult
+	 * @param querySet
+	 * @return
+	 */
+	protected Thread executeRunner(String name, DataModelResult dataModelResult, QueryResult queryResult, QuerySet querySet) {
+		ThreadTime threadTime = new ThreadTime();
+		queryResult.getThreadTimes().add(threadTime);
+		threadTime.setThreadName(name);
+		queryResult.setHint(this.queryHint);
+		logger.info("\nExecuting query "
+				+ queryResult.getStatement());
+		Thread thread;
+		if (this.runMode == RunMode.FUNCTIONAL) {
+			thread = new MultithreadedDiffer(
+					threadTime.getThreadName(),
+					queryResult,
+					threadTime, querySet.getNumberOfExecutions(), querySet.getExecutionDurationInMs())
+					.start();
+		} else {
+			thread = new MultithreadedRunner(
+					threadTime.getThreadName(),
+					queryResult,
+					dataModelResult,
+					threadTime, querySet.getNumberOfExecutions(), querySet.getExecutionDurationInMs())
+					.start();
+		}
+		return thread;
+	}
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/QueryVerifier.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/QueryVerifier.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/QueryVerifier.java
new file mode 100644
index 0000000..78f18ca
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/QueryVerifier.java
@@ -0,0 +1,195 @@
+/*
+ * 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.workload;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.FileReader;
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.phoenix.pherf.result.file.Extension;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.phoenix.pherf.PherfConstants;
+import org.apache.phoenix.pherf.configuration.Query;
+import org.apache.phoenix.pherf.util.PhoenixUtil;
+
+import difflib.DiffUtils;
+import difflib.Patch;
+
+public class QueryVerifier {
+	private PhoenixUtil pUtil = new PhoenixUtil();
+	private static final Logger logger = LoggerFactory
+			.getLogger(QueryVerifier.class);
+	private boolean useTemporaryOutput;
+	private String directoryLocation;
+
+	public QueryVerifier(boolean useTemporaryOutput) {
+		this.useTemporaryOutput = useTemporaryOutput;
+		this.directoryLocation = this.useTemporaryOutput ? 
+				PherfConstants.EXPORT_TMP : PherfConstants.EXPORT_DIR;
+		
+		ensureBaseDirExists();
+	}
+	
+	/***
+	 * Export query resultSet to CSV file
+	 * @param query
+	 * @throws Exception
+	 */
+	public String exportCSV(Query query) throws Exception {
+		Connection conn = null;
+		PreparedStatement statement = null;
+		ResultSet rs = null;
+		String fileName = getFileName(query);
+		FileOutputStream fos = new FileOutputStream(fileName);
+		try {
+			conn = pUtil.getConnection(query.getTenantId());
+			statement = conn.prepareStatement(query.getStatement());
+			boolean isQuery = statement.execute();
+			if (isQuery) {
+				rs = statement.executeQuery();
+				int columnCount = rs.getMetaData().getColumnCount();
+				while (rs.next()) {
+					for (int columnNum = 1; columnNum <= columnCount; columnNum++) {
+						fos.write((rs.getString(columnNum) + PherfConstants.RESULT_FILE_DELIMETER).getBytes());
+					}
+					fos.write(PherfConstants.NEW_LINE.getBytes());
+				}
+			} else {
+				conn.commit();
+			}
+		} catch (Exception e) {
+			e.printStackTrace();
+		} finally {
+			if (rs != null) rs.close();
+			if (statement != null) statement.close();
+			if (conn != null) conn.close();
+			fos.flush();
+			fos.close();
+		}
+		return fileName;
+	}
+	
+	/***
+	 * Do a diff between exported query results and temporary CSV file
+	 * @param query
+	 * @param newCSV
+	 * @return
+	 */
+	public boolean doDiff(Query query, String newCSV) {
+        List<String> original = fileToLines(getCSVName(query, PherfConstants.EXPORT_DIR, ""));
+        List<String> newLines  = fileToLines(newCSV);
+        
+        Patch patch = DiffUtils.diff(original, newLines);
+        if (patch.getDeltas().isEmpty()) {
+        	logger.info("Match: " + query.getId() + " with " + newCSV);
+        	return true;
+        } else {
+        	logger.error("DIFF FAILED: " + query.getId() + " with " + newCSV);
+        	return false;
+        }
+	}
+	
+	/***
+	 * Helper method to load file
+	 * @param filename
+	 * @return
+	 */
+    private static List<String> fileToLines(String filename) {
+            List<String> lines = new LinkedList<String>();
+            String line = "";
+            try {
+                    BufferedReader in = new BufferedReader(new FileReader(filename));
+                    while ((line = in.readLine()) != null) {
+                            lines.add(line);
+                    }
+                    in.close();
+            } catch (IOException e) {
+                    e.printStackTrace();
+            }
+            
+            return lines;
+    }
+
+    /**
+     * Get explain plan for a query
+     * @param query
+     * @return
+     * @throws SQLException
+     */
+	public String getExplainPlan(Query query) throws SQLException {
+		Connection conn = null;
+		ResultSet rs = null;
+		PreparedStatement statement = null;
+		StringBuilder buf = new StringBuilder();
+		try {
+			conn = pUtil.getConnection(query.getTenantId());
+			statement = conn.prepareStatement("EXPLAIN " + query.getStatement());
+			rs = statement.executeQuery();
+	        while (rs.next()) {
+	            buf.append(rs.getString(1).trim().replace(",", "-"));
+	        }
+			statement.close();
+		} catch (Exception e) {
+			e.printStackTrace();
+		} finally {
+			if (rs != null) rs.close();
+			if (statement != null) statement.close();
+			if (conn != null) conn.close();
+		}
+		return buf.toString();
+	}
+	
+    /***
+     * Helper method to generate CSV file name
+     * @param query
+     * @return
+     * @throws FileNotFoundException
+     */
+	private String getFileName(Query query) throws FileNotFoundException {
+		String tempExt = "";
+		if (this.useTemporaryOutput) {
+			tempExt = "_" + java.util.UUID.randomUUID().toString();
+		}
+		return getCSVName(query, this.directoryLocation, tempExt);
+	}
+	
+	private String getCSVName(Query query, String directory, String tempExt) {
+		String csvFile = directory + PherfConstants.PATH_SEPARATOR
+		        + query.getId() + tempExt + Extension.CSV.toString();
+				return csvFile;
+	}
+	
+	private void ensureBaseDirExists() {
+		File baseDir = new File(this.directoryLocation);
+		if (!baseDir.exists()) {
+			baseDir.mkdir();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/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
new file mode 100644
index 0000000..5cc6515
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/WorkloadExecutor.java
@@ -0,0 +1,115 @@
+/*
+ * 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.workload;
+
+import org.apache.phoenix.pherf.PherfConstants;
+import org.apache.phoenix.pherf.PherfConstants.RunMode;
+import org.apache.phoenix.pherf.configuration.XMLConfigParser;
+import org.apache.phoenix.pherf.jmx.MonitorManager;
+import org.apache.phoenix.pherf.loaddata.DataLoader;
+
+import org.apache.phoenix.pherf.util.ResourceList;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Properties;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+public class WorkloadExecutor {
+    private static final Logger logger = LoggerFactory.getLogger(WorkloadExecutor.class);
+    private final XMLConfigParser parser;
+    private MonitorManager monitor;
+    private Future monitorThread;
+    private final Properties properties;
+    private final int poolSize;
+
+    private final ExecutorService pool;
+
+
+    public WorkloadExecutor() throws Exception {
+        this(new ResourceList().getProperties());
+    }
+
+    public WorkloadExecutor(Properties properties) throws Exception{
+        this(properties,PherfConstants.DEFAULT_FILE_PATTERN);
+    }
+
+    public WorkloadExecutor(Properties properties, String filePattern) throws Exception {
+        this(properties,
+                new XMLConfigParser(filePattern),
+                true);
+    }
+
+    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"));
+
+        this.pool = Executors.newFixedThreadPool(this.poolSize);
+        if (monitor) {
+            initMonitor(Integer.parseInt(properties.getProperty("pherf.default.monitorFrequency")));
+        }
+    }
+
+    /**
+     * Executes all scenarios dataload
+     *
+     * @throws Exception
+     */
+    public void executeDataLoad() throws Exception {
+        logger.info("\n\nStarting Data Loader...");
+        DataLoader dataLoader = new DataLoader(properties, parser);
+        dataLoader.execute();
+    }
+
+    /**
+     * Executes all scenario multi-threaded query sets
+     *
+     * @param queryHint
+     * @throws Exception
+     */
+    public void executeMultithreadedQueryExecutor(String queryHint, boolean export, RunMode runMode) throws Exception {
+        logger.info("\n\nStarting Query Executor...");
+        QueryExecutor queryExecutor = new QueryExecutor(parser);
+        queryExecutor.execute(queryHint, export, runMode);
+    }
+
+    public void shutdown() throws Exception {
+		if (null != monitor && monitor.isRunning()) {
+            this.monitor.stop();
+            this.monitorThread.get(60, TimeUnit.SECONDS);
+            this.pool.shutdown();
+        }
+    }
+
+    // Just used for testing
+    public XMLConfigParser getParser() {
+        return parser;
+    }
+
+    private void initMonitor(int monitorFrequency) throws Exception {
+        this.monitor = new MonitorManager(monitorFrequency);
+        monitorThread = pool.submit(this.monitor);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/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
new file mode 100644
index 0000000..ec2f88d
--- /dev/null
+++ b/phoenix-pherf/src/main/resources/datamodel/create_prod_test_unsalted.sql
@@ -0,0 +1,33 @@
+/*
+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.PHERF_PROD_TEST_UNSALTED (
+    TENANT_ID CHAR(15) NOT NULL,
+    CREATED_DATE DATE NOT NULL,
+    FIELD VARCHAR,
+    DATA_TYPE VARCHAR,
+    OLDVAL_STRING VARCHAR,
+    NEWVAL_STRING VARCHAR,
+    DIVISION INTEGER,
+    CONNECTION_ID VARCHAR   
+    CONSTRAINT PK PRIMARY KEY 
+    (
+        TENANT_ID,
+        CREATED_DATE DESC,
+    )
+) VERSIONS=1,MULTI_TENANT=true

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/resources/hbase-site.xml
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/resources/hbase-site.xml b/phoenix-pherf/src/main/resources/hbase-site.xml
new file mode 100644
index 0000000..8b7d0db
--- /dev/null
+++ b/phoenix-pherf/src/main/resources/hbase-site.xml
@@ -0,0 +1,25 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+  ~ 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.
+  -->
+<configuration>
+  <property>
+    <name>phoenix.query.threadPoolSize</name>
+    <value>128</value>
+  </property>
+</configuration>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/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
new file mode 100644
index 0000000..5e20bf0
--- /dev/null
+++ b/phoenix-pherf/src/main/resources/scenario/prod_test_unsalted_scenario.xml
@@ -0,0 +1,342 @@
+<?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="194" 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 -->
+            <type>VARCHAR</type>
+            <dataSequence>RANDOM</dataSequence>
+            <length>15</length>
+            <name>GENERAL_VARCHAR</name>
+        </column>
+        <column>
+            <type>CHAR</type>
+            <dataSequence>RANDOM</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>90</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>CHAR</type>
+            <userDefined>true</userDefined>
+            <dataSequence>LIST</dataSequence>
+            <length>15</length>
+            <name>TENANT_ID</name>
+            <valuelist>
+                <datavalue distribution="40">
+                    <value>00Dxx0000001gER</value>
+                </datavalue>
+                <datavalue distribution="20">
+                    <value>00Dxx0000001gES</value>
+                </datavalue>
+                <datavalue distribution="20">
+                    <value>00Dxx0000001gET</value>
+                </datavalue>
+                <datavalue distribution="15">
+                    <value>00Dxx0000001gEU</value>
+                </datavalue>
+                <datavalue distribution="5">
+                    <value>00Dxx0000001gEV</value>
+                </datavalue>
+            </valuelist>
+        </column>
+        <column>
+            <type>DATE</type>
+            <userDefined>true</userDefined>
+            <dataSequence>LIST</dataSequence>
+            <name>CREATED_DATE</name>
+            <nullChance>0</nullChance>
+            <valuelist>
+                <datavalue distribution="2">
+                    <minValue>2014-08-31 00:00:00.000</minValue>
+                    <maxValue>2014-09-01 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-09-01 00:00:00.000</minValue>
+                    <maxValue>2014-09-02 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-09-02 00:00:00.000</minValue>
+                    <maxValue>2014-09-03 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-09-03 00:00:00.000</minValue>
+                    <maxValue>2014-09-04 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-09-04 00:00:00.000</minValue>
+                    <maxValue>2014-09-05 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-09-05 00:00:00.000</minValue>
+                    <maxValue>2014-09-06 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-09-06 00:00:00.000</minValue>
+                    <maxValue>2014-09-07 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-09-07 00:00:00.000</minValue>
+                    <maxValue>2014-09-08 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-09-08 00:00:00.000</minValue>
+                    <maxValue>2014-09-09 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-09-09 00:00:00.000</minValue>
+                    <maxValue>2014-09-10 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-09-10 00:00:00.000</minValue>
+                    <maxValue>2014-09-11 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-09-11 00:00:00.000</minValue>
+                    <maxValue>2014-09-12 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-09-12 00:00:00.000</minValue>
+                    <maxValue>2014-09-13 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-09-13 00:00:00.000</minValue>
+                    <maxValue>2014-09-14 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-09-14 00:00:00.000</minValue>
+                    <maxValue>2014-09-15 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-09-15 00:00:00.000</minValue>
+                    <maxValue>2014-09-16 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-09-16 00:00:00.000</minValue>
+                    <maxValue>2014-09-17 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-09-17 00:00:00.000</minValue>
+                    <maxValue>2014-09-18 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-09-18 00:00:00.000</minValue>
+                    <maxValue>2014-09-19 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-09-19 00:00:00.000</minValue>
+                    <maxValue>2014-09-20 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-09-20 00:00:00.000</minValue>
+                    <maxValue>2014-09-21 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-09-21 00:00:00.000</minValue>
+                    <maxValue>2014-09-22 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-09-22 00:00:00.000</minValue>
+                    <maxValue>2014-09-23 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-09-23 00:00:00.000</minValue>
+                    <maxValue>2014-09-24 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-09-24 00:00:00.000</minValue>
+                    <maxValue>2014-09-25 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-09-25 00:00:00.000</minValue>
+                    <maxValue>2014-09-26 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-09-26 00:00:00.000</minValue>
+                    <maxValue>2014-09-27 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-09-27 00:00:00.000</minValue>
+                    <maxValue>2014-09-28 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-09-28 00:00:00.000</minValue>
+                    <maxValue>2014-09-29 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-09-29 00:00:00.000</minValue>
+                    <maxValue>2014-09-30 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-09-30 00:00:00.000</minValue>
+                    <maxValue>2014-10-01 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-10-01 00:00:00.000</minValue>
+                    <maxValue>2014-10-02 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-10-02 00:00:00.000</minValue>
+                    <maxValue>2014-10-03 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-10-03 00:00:00.000</minValue>
+                    <maxValue>2014-10-04 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-10-04 00:00:00.000</minValue>
+                    <maxValue>2014-10-05 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-10-05 00:00:00.000</minValue>
+                    <maxValue>2014-10-06 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-10-06 00:00:00.000</minValue>
+                    <maxValue>2014-10-07 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-10-07 00:00:00.000</minValue>
+                    <maxValue>2014-10-08 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-10-08 00:00:00.000</minValue>
+                    <maxValue>2014-10-09 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-10-09 00:00:00.000</minValue>
+                    <maxValue>2014-10-10 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-10-10 00:00:00.000</minValue>
+                    <maxValue>2014-10-11 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-10-11 00:00:00.000</minValue>
+                    <maxValue>2014-10-12 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-10-12 00:00:00.000</minValue>
+                    <maxValue>2014-10-13 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-10-13 00:00:00.000</minValue>
+                    <maxValue>2014-10-14 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-10-14 00:00:00.000</minValue>
+                    <maxValue>2014-10-15 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-10-15 00:00:00.000</minValue>
+                    <maxValue>2014-10-16 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-10-16 00:00:00.000</minValue>
+                    <maxValue>2014-10-17 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-10-17 00:00:00.000</minValue>
+                    <maxValue>2014-10-18 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-10-18 00:00:00.000</minValue>
+                    <maxValue>2014-10-19 00:00:00.000</maxValue>
+                </datavalue>
+                <datavalue distribution="2">
+                    <minValue>2014-10-19 00:00:00.000</minValue>
+                    <maxValue>2014-10-20 00:00:00.000</maxValue>
+                </datavalue>
+            </valuelist>
+        </column>
+    </datamapping>
+    <scenarios>
+        <scenario tableName="PHERF.PHERF_PROD_TEST_UNSALTED" rowCount="10000">
+            <!-- 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>LIST</dataSequence>
+                    <valueList>
+                        <datavalue>
+                            <value>00Dxx0000001gER</value>
+                        </datavalue>
+                        <datavalue>
+                            <value>00Dxx0000001gES</value>
+                        </datavalue>
+                        <datavalue>
+                            <value>00Dxx0000001gET</value>
+                        </datavalue>
+                    </valueList>
+                    <name>TENANT_ID</name>
+                </column>
+            </dataOverride>
+            <!--Minimum of executionDurationInMs or numberOfExecutions. Which ever is reached first -->
+            <querySet concurrency="1" executionType="PARALLEL" executionDurationInMs="60000" numberOfExecutions="100">
+                <!--  Aggregate queries on a per tenant basis -->
+                <query tenantId="00Dxx0000001gER"
+                       ddl="CREATE VIEW IF NOT EXISTS PHERF.PHERF_TEST_VIEW_UNSALTED AS SELECT * FROM PHERF.PHERF_PROD_TEST_UNSALTED"
+                       statement="select count(*) from PHERF.PHERF_TEST_VIEW_UNSALTED"/>
+                <query tenantId="00Dxx0000001gES"
+                       ddl="CREATE VIEW IF NOT EXISTS PHERF.PHERF_TEST_VIEW_UNSALTED AS SELECT * FROM PHERF.PHERF_PROD_TEST_UNSALTED"
+                       statement="select /*+ SMALL*/ count(*) from PHERF.PHERF_TEST_VIEW_UNSALTED"/>
+            </querySet>
+
+        </scenario>
+    </scenarios>
+</datamodel>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/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
new file mode 100644
index 0000000..d4f3633
--- /dev/null
+++ b/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/BaseTestWithCluster.java
@@ -0,0 +1,67 @@
+/*
+ * 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);
+    }
+
+    /**
+     * Get the configuration for what scenarios will run and how.
+     *
+     * @return {@link java.nio.file.Path}
+     */
+    public Path getTestConfiguration() {
+        URL resourceUrl = getUrl();
+        assertNotNull("Test data XML file is missing", resourceUrl);
+        Path resourcePath = null;
+        try {
+            resourcePath = Paths.get(resourceUrl.toURI());
+        } catch (URISyntaxException e) {
+            e.printStackTrace();
+        }
+        return resourcePath;
+    }
+
+    public URL getUrl() {
+        return getClass().getResource("/scenario/test_scenario.xml");
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ColumnTest.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ColumnTest.java b/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ColumnTest.java
new file mode 100644
index 0000000..e573c07
--- /dev/null
+++ b/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ColumnTest.java
@@ -0,0 +1,50 @@
+/*
+ * 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.junit.Test;
+
+import static org.junit.Assert.assertTrue;
+
+public class ColumnTest {
+    @Test
+    public void testColumnMutate() {
+        Column columnA = new Column();
+        Column columnB = new Column();
+        Column columnC = new Column();
+        columnA.setType(DataTypeMapping.VARCHAR);
+        columnB.setType(DataTypeMapping.VARCHAR);
+        columnA.setLength(15);
+        columnA.setMinValue(20);
+        columnA.setMaxValue(25);
+        columnB.setLength(30);
+        columnC.setMaxValue(45);
+
+        columnA.mutate(columnB);
+        assertTrue("Mutation failed length", columnA.getLength() == columnB.getLength());
+        columnA.mutate(columnC);
+        assertTrue("Mutation failed length", columnA.getLength() == columnB.getLength());
+        assertTrue("Mutation failed min", columnA.getMinValue() == 20);
+        assertTrue("Mutation failed max", columnA.getMaxValue() == columnC.getMaxValue());
+        assertTrue("Mutation failed name", columnA.getName() == null);
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/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
new file mode 100644
index 0000000..769bdc0
--- /dev/null
+++ b/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ConfigurationParserTest.java
@@ -0,0 +1,200 @@
+/*
+ * 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 java.net.URL;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+
+import org.apache.phoenix.pherf.configuration.*;
+import org.apache.phoenix.pherf.rules.DataValue;
+import org.apache.phoenix.pherf.workload.WorkloadExecutor;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.JAXBException;
+import javax.xml.bind.Marshaller;
+
+import static org.junit.Assert.*;
+
+public class ConfigurationParserTest {
+    private static final Logger logger = LoggerFactory.getLogger(ConfigurationParserTest.class);
+
+    @Test
+    public void testConfigFilesParsing() {
+        try {
+        	WorkloadExecutor workloadExec = new WorkloadExecutor();
+            List<Scenario> scenarioList = workloadExec.getParser().getScenarios();
+            assertTrue("Could not load the scenarios from xml.", (scenarioList != null) && (scenarioList.size() > 0));
+            logger.info("Number of scenarios loaded: " + scenarioList.size());
+
+        } catch (Exception e) {
+            e.printStackTrace();
+            fail();
+        }
+    }
+
+	@Test
+    // TODO Break this into multiple smaller tests.
+    public void testConfigReader(){
+		URL resourceUrl = getClass().getResource("/scenario/test_scenario.xml");
+        assertNotNull("Test data XML file is missing", resourceUrl);
+
+		try {
+//            writeXML();
+			Path resourcePath = Paths.get(resourceUrl.toURI());
+            DataModel data = XMLConfigParser.readDataModel(resourcePath);
+            List<Scenario> scenarioList = data.getScenarios();
+            assertTrue("Could not load the scenarios from xml.", (scenarioList != null) && (scenarioList.size() > 0));
+            List<Column> dataMappingColumns = data.getDataMappingColumns();
+            assertTrue("Could not load the data columns from xml.", (dataMappingColumns != null) && (dataMappingColumns.size() > 0));
+            assertTrue("Could not load the data DataValue list from xml.",
+                    (dataMappingColumns.get(6).getDataValues() != null)
+                    && (dataMappingColumns.get(6).getDataValues().size() > 0));
+
+            assertDateValue(dataMappingColumns);
+
+            // Validate column mappings
+            for (Column column : dataMappingColumns) {
+                assertNotNull("Column ("+ column.getName() + ") is missing its type",column.getType());
+            }
+
+            Scenario scenario = scenarioList.get(0);
+            assertNotNull(scenario);
+            assertEquals("PHERF.TEST_TABLE", scenario.getTableName());
+            assertEquals(50, scenario.getRowCount());
+            assertEquals(1, scenario.getDataOverride().getColumn().size());
+            QuerySet qs = scenario.getQuerySet().get(0);
+            assertEquals(ExecutionType.SERIAL, qs.getExecutionType());
+            assertEquals(5000, qs.getExecutionDurationInMs());
+            assertEquals(2, qs.getQuery().size());
+
+            Query firstQuery = qs.getQuery().get(0);
+            assertEquals("1-3", qs.getConcurrency());
+            assertEquals(1, qs.getMinConcurrency());
+            assertEquals(3, qs.getMaxConcurrency());
+            assertEquals(100, qs.getNumberOfExecutions());
+            assertEquals("select count(*) from PHERF.TEST_TABLE", firstQuery.getStatement());
+            assertEquals("123456789012345", firstQuery.getTenantId());
+            assertEquals(null, firstQuery.getDdl());
+            assertEquals(0, (long)firstQuery.getExpectedAggregateRowCount());
+
+            Query secondQuery = qs.getQuery().get(1);
+            assertEquals("Could not get statement.", "select sum(SOME_INT) from PHERF.TEST_TABLE", secondQuery.getStatement());
+            assertEquals("Could not get queryGroup.", "g1", secondQuery.getQueryGroup());
+
+            // Make sure anything in the overrides matches a real column in the data mappings
+            DataOverride override = scenario.getDataOverride();
+            for (Column column : override.getColumn()) {
+                assertTrue("Could not lookup Column (" + column.getName() + ") in DataMapping columns: " + dataMappingColumns, dataMappingColumns.contains(column));
+            }
+
+		} catch (Exception e) {
+			e.printStackTrace();
+			fail();
+		}
+	}
+
+    private void assertDateValue(List<Column> dataMappingColumns) {
+        for (Column dataMapping : dataMappingColumns) {
+            if ((dataMapping.getType() == DataTypeMapping.DATE) && (dataMapping.getName().equals("CREATED_DATE"))) {
+                // First rule should have min/max set
+                assertNotNull(dataMapping.getDataValues().get(0).getMinValue());
+                assertNotNull(dataMapping.getDataValues().get(0).getMaxValue());
+
+                // Second rule should have only value set
+                assertNotNull(dataMapping.getDataValues().get(1).getValue());
+
+                // Third rule should have min/max set
+                assertNotNull(dataMapping.getDataValues().get(2).getMinValue());
+                assertNotNull(dataMapping.getDataValues().get(2).getMaxValue());
+                return;
+            }
+        }
+        fail("We should have found a Rule value that matched.");
+    }
+
+    /*
+        Used for debugging to dump out a simple xml filed based on the bound objects.
+     */
+	private void writeXML() {
+        try {
+            DataValue dataValue = new DataValue();
+            dataValue.setDistribution(20);
+            dataValue.setValue("jnhgGhHminwiajn");
+            List<DataValue> dataValueList = new ArrayList<>();
+            dataValueList.add(dataValue);
+            Column column = new Column();
+            column.setLength(15);
+            column.setDataSequence(DataSequence.RANDOM);
+            column.setName("TEST_COL");
+            column.setUserDefined(true);
+            column.setDataValues(dataValueList);
+            List<Column> columnList = new ArrayList<>();
+            columnList.add(column);
+
+            DataModel data = new DataModel();
+            data.setRelease("192");
+            data.setDataMappingColumns(columnList);
+
+            Scenario scenario = new Scenario();
+            scenario.setPhoenixProperties(new HashMap<String, String>());
+            scenario.getPhoenixProperties().put("phoenix.query.threadPoolSize", "200");
+            scenario.setDataOverride(new DataOverride());
+            scenario.setTableName("tableName");
+            scenario.setRowCount(10);
+            QuerySet querySet = new QuerySet();
+            querySet.setExecutionType(ExecutionType.PARALLEL);
+            querySet.setExecutionDurationInMs(10000);
+            scenario.getQuerySet().add(querySet);
+            Query query = new Query();
+            querySet.getQuery().add(query);
+            querySet.setConcurrency("15");
+            querySet.setNumberOfExecutions(20);
+            query.setStatement("select * from FHA");
+            Scenario scenario2 = new Scenario();
+            scenario2.setPhoenixProperties(new HashMap<String, String>());
+            scenario2.setDataOverride(new DataOverride());
+            scenario2.setTableName("tableName2");
+            scenario2.setRowCount(500);
+            List<Scenario> scenarios = new ArrayList<Scenario>();
+            scenarios.add(scenario);
+            scenarios.add(scenario2);
+            data.setScenarios(scenarios);
+
+            // create JAXB context and initializing Marshaller
+            JAXBContext jaxbContext = JAXBContext.newInstance(DataModel.class);
+            Marshaller jaxbMarshaller = jaxbContext.createMarshaller();
+
+            // for getting nice formatted output
+            jaxbMarshaller.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, Boolean.TRUE);
+
+            // Writing to console
+            jaxbMarshaller.marshal(data, System.out);
+        } catch (JAXBException e) {
+            // some exception occured
+            e.printStackTrace();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/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
new file mode 100644
index 0000000..0fc0dd9
--- /dev/null
+++ b/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/DataIngestTest.java
@@ -0,0 +1,78 @@
+/*
+ * 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/36b88651/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
new file mode 100644
index 0000000..b3d3946
--- /dev/null
+++ b/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/DataLoaderTest.java
@@ -0,0 +1,108 @@
+/*
+ * 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 {
+    private static XMLConfigParser parser = null;
+    private static DataModel model = null;
+    private DataLoader loader = null;
+
+    @BeforeClass
+    public static void init() {
+
+        try {
+            parser = new XMLConfigParser(matcherScenario);
+            model = parser.getDataModels().get(0);
+        } catch (Exception e) {
+            fail("Failed to initialize test: " + e.getMessage());
+        }
+    }
+
+    /**
+     * 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/36b88651/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
new file mode 100644
index 0000000..f15b4d4
--- /dev/null
+++ b/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/PherfTest.java
@@ -0,0 +1,67 @@
+/*
+ * 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.junit.Rule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.ExpectedSystemExit;
+
+import java.sql.Date;
+
+public class PherfTest extends BaseTestWithCluster {
+    @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"};
+
+        // Makes sure that System.exit(1) is called. Release is a required param.
+        exit.expectSystemExitWithStatus(1);
+        Pherf.main(args);
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ResourceTest.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ResourceTest.java b/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ResourceTest.java
new file mode 100644
index 0000000..042f9c3
--- /dev/null
+++ b/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ResourceTest.java
@@ -0,0 +1,68 @@
+/*
+ * 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.ResourceList;
+import org.apache.phoenix.pherf.PherfConstants;
+import org.junit.Test;
+
+import java.nio.file.Path;
+import java.util.Collection;
+import java.util.Properties;
+
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertThat;
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.junit.Assert.assertTrue;
+
+public class ResourceTest {
+    @Test
+    public void testSchemaResourceList() throws Exception{
+        String extension = ".sql";
+        assertResources(PherfConstants.SCHEMA_ROOT_PATTERN + extension, PherfConstants.RESOURCE_DATAMODEL, extension);
+    }
+
+    @Test
+    public void testScenarioResourceList() throws Exception {
+        String extension = ".xml";
+        assertResources(PherfConstants.SCENARIO_ROOT_PATTERN + extension, PherfConstants.RESOURCE_SCENARIO, extension);
+    }
+
+    @Test
+    public void testResourceListPropertyDirectory() throws Exception {
+
+        ResourceList list = new ResourceList();
+        Properties properties = list.getProperties();
+        assertTrue("Property file list was empty", properties.size() > 0);
+        assertNotNull(properties.getProperty("pherf.default.dataloader.threadpool"));
+    }
+
+    private Collection<Path> assertResources(String pattern, String rootDir, String assertStr) throws Exception {
+        ResourceList list = new ResourceList(rootDir);
+        Collection<Path> paths =
+                list.getResourceList(pattern);
+        assertTrue("Resource file list was empty", paths.size() > 0);
+        for (Path path : paths) {
+            assertThat(path.toString(), containsString(assertStr));
+        }
+        return paths;
+    }
+}
\ No newline at end of file


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

Posted by mu...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/XMLConfigParser.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/XMLConfigParser.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/XMLConfigParser.java
new file mode 100644
index 0000000..e9255c6
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/configuration/XMLConfigParser.java
@@ -0,0 +1,157 @@
+/*
+ * 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.PherfConstants;
+import org.apache.phoenix.pherf.exception.FileLoaderException;
+import org.apache.phoenix.pherf.util.ResourceList;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.JAXBException;
+import javax.xml.bind.Marshaller;
+import javax.xml.bind.Unmarshaller;
+import java.io.OutputStream;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+public class XMLConfigParser {
+
+    private static final Logger logger = LoggerFactory.getLogger(XMLConfigParser.class);
+    private String filePattern;
+    private List<DataModel> dataModels;
+    private List<Scenario> scenarios = null;
+    private ResourceList resourceList;
+    private Collection<Path> paths = null;
+
+    public XMLConfigParser(String pattern) throws Exception {
+        init(pattern);
+    }
+
+    public List<DataModel> getDataModels() {
+        return dataModels;
+    }
+
+    public synchronized Collection<Path> getPaths(String strPattern) throws Exception {
+        if (paths != null) {
+            return paths;
+        }
+        paths = getResources(strPattern);
+        return paths;
+    }
+
+    public synchronized List<Scenario> getScenarios() throws Exception {
+        if (scenarios != null) {
+            return scenarios;
+        }
+
+        scenarios = (List<Scenario>) Collections.synchronizedCollection(new ArrayList<Scenario>());
+        for (Path path : getPaths(getFilePattern())) {
+            try {
+                List<Scenario> scenarioList = XMLConfigParser.readDataModel(path).getScenarios();
+                for (Scenario scenario : scenarioList) {
+                    scenarios.add(scenario);
+                }
+            } catch (JAXBException e) {
+                e.printStackTrace();
+            }
+        }
+        return scenarios;
+    }
+
+    public String getFilePattern() {
+        return filePattern;
+    }
+
+    /**
+     * Unmarshall an XML data file
+     *
+     * @param file Name of File
+     * @return
+     * @throws JAXBException
+     */
+    // TODO Remove static calls
+    public static DataModel readDataModel(Path file) throws JAXBException {
+        JAXBContext jaxbContext = JAXBContext.newInstance(DataModel.class);
+        Unmarshaller jaxbUnmarshaller = jaxbContext.createUnmarshaller();
+        String fName = PherfConstants.RESOURCE_SCENARIO + "/" + file.getFileName().toString();
+        logger.info("Open config file: " + fName);
+        return (DataModel) jaxbUnmarshaller
+                .unmarshal(XMLConfigParser.class.getResourceAsStream(fName));
+    }
+
+    // TODO Remove static calls
+    public static String parseSchemaName(String fullTableName) {
+        String ret = null;
+        if (fullTableName.contains(".")) {
+            ret = fullTableName.substring(0, fullTableName.indexOf("."));
+        }
+        return ret;
+    }
+
+    // TODO Remove static calls
+    public static String parseTableName(String fullTableName) {
+        String ret = fullTableName;
+        if (fullTableName.contains(".")) {
+            ret = fullTableName.substring(fullTableName.indexOf(".") + 1, fullTableName.length());
+        }
+        return ret;
+    }
+
+    // TODO Remove static calls
+    public static void writeDataModel(DataModel data, OutputStream output) throws JAXBException {
+        // create JAXB context and initializing Marshaller
+        JAXBContext jaxbContext = JAXBContext.newInstance(DataModel.class);
+        Marshaller jaxbMarshaller = jaxbContext.createMarshaller();
+
+        // for getting nice formatted output
+        jaxbMarshaller.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, Boolean.TRUE);
+
+        // Writing to console
+        jaxbMarshaller.marshal(data, output);
+    }
+
+    private void init(String pattern) throws Exception {
+        if (dataModels != null) {
+            return;
+        }
+        this.filePattern = pattern;
+        this.dataModels = new ArrayList<>();
+        this.resourceList = new ResourceList(PherfConstants.RESOURCE_SCENARIO);
+        this.paths = getResources(this.filePattern);
+        if (this.paths.isEmpty()) {
+            throw new FileLoaderException(
+                    "Could not load the resource files using the pattern: " + pattern);
+        }
+        for (Path path : this.paths) {
+            System.out.println("Adding model for path:" + path.toString());
+            this.dataModels.add(XMLConfigParser.readDataModel(path));
+        }
+    }
+
+    private Collection<Path> getResources(String pattern) throws Exception {
+        Collection<Path> resourceFiles = new ArrayList<Path>();
+        resourceFiles = resourceList.getResourceList(pattern);
+        return resourceFiles;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/exception/FileLoaderException.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/exception/FileLoaderException.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/exception/FileLoaderException.java
new file mode 100644
index 0000000..63784eb
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/exception/FileLoaderException.java
@@ -0,0 +1,28 @@
+/*
+ * 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.exception;
+
+public class FileLoaderException extends PherfException {
+    public FileLoaderException(String message) throws Exception {
+        super(message);
+    }
+    public FileLoaderException(String message, Exception e) {
+        super(message, e);
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/exception/FileLoaderRuntimeException.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/exception/FileLoaderRuntimeException.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/exception/FileLoaderRuntimeException.java
new file mode 100644
index 0000000..4ab751b
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/exception/FileLoaderRuntimeException.java
@@ -0,0 +1,28 @@
+/*
+ * 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.exception;
+
+public class FileLoaderRuntimeException extends PherfRuntimeException {
+    public FileLoaderRuntimeException(String message) throws Exception {
+        super(message);
+    }
+    public FileLoaderRuntimeException(String message, Exception e) {
+        super(message, e);
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/exception/PherfException.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/exception/PherfException.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/exception/PherfException.java
new file mode 100644
index 0000000..1748b26
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/exception/PherfException.java
@@ -0,0 +1,30 @@
+/*
+ * 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.exception;
+
+public class PherfException extends Exception {
+    public PherfException(String message) throws Exception{
+        super(message);
+    }
+
+    public PherfException(String message, Exception e) {
+        super(message, e);
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/exception/PherfRuntimeException.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/exception/PherfRuntimeException.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/exception/PherfRuntimeException.java
new file mode 100644
index 0000000..e12feba
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/exception/PherfRuntimeException.java
@@ -0,0 +1,30 @@
+/*
+ * 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.exception;
+
+public class PherfRuntimeException extends RuntimeException {
+    public PherfRuntimeException(String message) throws Exception{
+        super(message);
+    }
+
+    public PherfRuntimeException(String message, Exception e) {
+        super(message, e);
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/MonitorDetails.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/MonitorDetails.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/MonitorDetails.java
new file mode 100644
index 0000000..0a25ca4
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/MonitorDetails.java
@@ -0,0 +1,50 @@
+/*
+ * 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.jmx;
+
+import org.apache.phoenix.pherf.jmx.monitors.*;
+
+public enum MonitorDetails {
+    FREE_MEMORY("org.apache.phoenix.pherf:type=RuntimeFreeMemory", new FreeMemoryMonitor()),
+    TOTAL_MEMORY("org.apache.phoenix.pherf:type=RuntimeTotalMemory", new TotalMemoryMonitor()),
+    MAX_MEMORY("org.apache.phoenix.pherf:type=RuntimeMaxMemory", new MaxMemoryMonitor()),
+    HEAP_MEMORY_USAGE("org.apache.phoenix.pherf:type=HeapMemoryUsage", new HeapMemoryMonitor()),
+    NON_HEAP_MEMORY_USAGE("org.apache.phoenix.pherf:type=NonHeapMemoryUsage", new NonHeapMemoryMonitor()),
+    OBJECT_PENDING_FINALIZATION("org.apache.phoenix.pherf:type=ObjectPendingFinalizationCount", new ObjectPendingFinalizationCountMonitor()),
+    GARBAGE_COLLECTOR_ELAPSED_TIME("org.apache.phoenix.pherf:type=GarbageCollectorElapsedTime", new GarbageCollectorElapsedTimeMonitor()),
+    CPU_LOAD_AVERAGE("org.apache.phoenix.pherf:type=CPULoadAverage", new CPULoadAverageMonitor()),
+    THREAD_COUNT("org.apache.phoenix.pherf:type=PherfThreads",new ThreadMonitor());
+
+    private final String monitorName;
+    private final Monitor monitor;
+
+    private MonitorDetails(String monitorName, Monitor monitor) {
+        this.monitorName = monitorName;
+        this.monitor = monitor;
+    }
+
+    @Override
+    public String toString() {
+        return monitorName;
+    }
+
+    public Monitor getMonitor() {
+        return monitor;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/MonitorManager.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/MonitorManager.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/MonitorManager.java
new file mode 100644
index 0000000..391db58
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/MonitorManager.java
@@ -0,0 +1,173 @@
+/*
+ * 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.jmx;
+
+import org.apache.phoenix.pherf.PherfConstants;
+import org.apache.phoenix.pherf.exception.FileLoaderRuntimeException;
+import org.apache.phoenix.pherf.jmx.monitors.Monitor;
+import org.apache.phoenix.pherf.result.file.ResultFileDetails;
+import org.apache.phoenix.pherf.result.impl.CSVResultHandler;
+import org.apache.phoenix.pherf.result.Result;
+import org.apache.phoenix.pherf.result.ResultHandler;
+import org.apache.phoenix.util.DateUtil;
+
+import javax.management.*;
+import java.io.IOException;
+import java.lang.management.ManagementFactory;
+import java.util.*;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * This class starts JMX stats for the configured monitors. Monitors should be configured in MonitorDetails Enum.
+ * Each stat implements {@link org.apache.phoenix.pherf.jmx.monitors.Monitor}.
+ *
+ * For the duration of any Pherf run, when the configured {@link org.apache.phoenix.pherf.PherfConstants#MONITOR_FREQUENCY}
+ * is reached a snapshot of each monitor is taken and dumped out to a log file.
+ */
+public class MonitorManager implements Runnable {
+    // List of MonitorDetails for all the running monitors.
+    // TODO Move this out to config. Possible use Guice and use IOC to inject it in.
+    private static final List<MonitorDetails> MONITOR_DETAILS_LIST =
+            Arrays.asList(MonitorDetails.values());
+    private final ResultHandler resultHandler;
+    private final long monitorFrequency;
+    private AtomicLong rowCount;
+    private volatile boolean shouldStop = false;
+    private volatile boolean isRunning = false;
+
+    public MonitorManager() throws Exception {
+        this(PherfConstants.MONITOR_FREQUENCY);
+    }
+
+    /**
+     *
+     * @param monitorFrequency Frequency at which monitor stats are written to a log file.
+     * @throws Exception
+     */
+    public MonitorManager(long monitorFrequency) throws Exception {
+        this.monitorFrequency = monitorFrequency;
+        MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
+
+        // Register all the monitors to JMX
+        for (MonitorDetails monitorDetails : MONITOR_DETAILS_LIST) {
+            StandardMBean bean = new StandardMBean(monitorDetails.getMonitor(), Monitor.class);
+            ObjectName monitorThreadStatName = new ObjectName(monitorDetails.toString());
+            try {
+                mbs.registerMBean(bean, monitorThreadStatName);
+            } catch (InstanceAlreadyExistsException e) {
+                mbs.unregisterMBean(monitorThreadStatName);
+                mbs.registerMBean(bean, monitorThreadStatName);
+            }
+        }
+        rowCount = new AtomicLong(0);
+        this.resultHandler = new CSVResultHandler(PherfConstants.MONITOR_FILE_NAME, ResultFileDetails.CSV);
+    }
+
+    @Override
+    public void run() {
+        try {
+            while (!shouldStop()) {
+                isRunning = true;
+                List rowValues = new ArrayList<String>();
+                synchronized (resultHandler) {
+                    for (MonitorDetails monitorDetails : MONITOR_DETAILS_LIST) {
+                        rowValues.clear();
+                        try {
+                            StandardMBean bean = new StandardMBean(monitorDetails.getMonitor(), Monitor.class);
+
+                            Calendar calendar = new GregorianCalendar();
+                            rowValues.add(monitorDetails);
+
+                            rowValues.add(((Monitor) bean.getImplementation()).getStat());
+                            rowValues.add(DateUtil.DEFAULT_MS_DATE_FORMATTER.format(calendar.getTime()));
+                            Result
+                                    result = new Result(ResultFileDetails.CSV, ResultFileDetails.CSV_MONITOR.getHeader().toString(), rowValues);
+                            resultHandler.write(result);
+                        } catch (Exception e) {
+                            throw new FileLoaderRuntimeException("Could not log monitor result.", e);
+                        }
+                        rowCount.getAndIncrement();
+                    }
+                    try {
+                        Thread.sleep(getMonitorFrequency());
+                    } catch (InterruptedException e) {
+                        Thread.currentThread().interrupt();
+                        e.printStackTrace();
+                    }
+                }
+            }
+        } finally {
+            try {
+                isRunning = false;
+                if (resultHandler != null) {
+                    resultHandler.flush();
+                    resultHandler.close();
+
+                }
+            } catch (Exception e) {
+                throw new FileLoaderRuntimeException("Could not close monitor results.", e);
+            }
+        }
+
+    }
+
+    public long getMonitorFrequency() {
+        return monitorFrequency;
+    }
+
+    public synchronized boolean shouldStop() {
+        return shouldStop;
+    }
+
+    public synchronized void stop() {
+        this.shouldStop = true;
+    }
+
+    public synchronized long getRowCount() {
+        return rowCount.get();
+    }
+
+    public synchronized boolean isRunning() {
+        return isRunning;
+    }
+
+    /**
+     * This method should really only be used for testing
+     *
+     * @return List < {@link org.apache.phoenix.pherf.result.Result} >
+     * @throws IOException
+     */
+    public synchronized List<Result> readResults() throws Exception {
+        ResultHandler handler = null;
+        try {
+            if (resultHandler.isClosed()) {
+                handler = new CSVResultHandler(PherfConstants.MONITOR_FILE_NAME, ResultFileDetails.CSV);
+                return handler.read();
+            } else {
+                return resultHandler.read();
+            }
+        } catch (Exception e) {
+            throw new FileLoaderRuntimeException("Could not close monitor results.", e);
+        } finally {
+            if (handler != null) {
+                handler.close();
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/Stat.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/Stat.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/Stat.java
new file mode 100644
index 0000000..ef3703c
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/Stat.java
@@ -0,0 +1,32 @@
+/*
+ * 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.jmx;
+
+public class Stat <T>{
+    private final T stat;
+
+    public Stat(T stat) {
+        this.stat = stat;
+    }
+
+    @Override
+    public String toString() {
+        return stat.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/CPULoadAverageMonitor.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/CPULoadAverageMonitor.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/CPULoadAverageMonitor.java
new file mode 100644
index 0000000..0823d43
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/CPULoadAverageMonitor.java
@@ -0,0 +1,33 @@
+/*
+ * 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.jmx.monitors;
+
+import org.apache.phoenix.pherf.jmx.Stat;
+
+import java.lang.management.ManagementFactory;
+import java.lang.management.MemoryMXBean;
+
+public class CPULoadAverageMonitor implements Monitor {
+
+    @Override
+    public Stat getStat() {
+        Stat<MemoryMXBean> stat = new Stat(ManagementFactory.getOperatingSystemMXBean().getSystemLoadAverage());
+        return stat;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/ExampleMonitor.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/ExampleMonitor.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/ExampleMonitor.java
new file mode 100644
index 0000000..1dd72e7
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/ExampleMonitor.java
@@ -0,0 +1,33 @@
+/*
+ * 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.jmx.monitors;
+
+import org.apache.phoenix.pherf.jmx.Stat;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+public class ExampleMonitor implements Monitor {
+    private final AtomicLong counter = new AtomicLong();
+
+    @Override
+    public Stat getStat() {
+        Stat<Long> stat = new Stat(new Long(counter.getAndIncrement()));
+        return stat;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/FreeMemoryMonitor.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/FreeMemoryMonitor.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/FreeMemoryMonitor.java
new file mode 100644
index 0000000..848863c
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/FreeMemoryMonitor.java
@@ -0,0 +1,30 @@
+/*
+ * 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.jmx.monitors;
+
+import org.apache.phoenix.pherf.jmx.Stat;
+
+public class FreeMemoryMonitor implements Monitor {
+
+    @Override
+    public Stat getStat() {
+        Stat<Long> stat = new Stat(new Long(Runtime.getRuntime().freeMemory()));
+        return stat;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/GarbageCollectorElapsedTimeMonitor.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/GarbageCollectorElapsedTimeMonitor.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/GarbageCollectorElapsedTimeMonitor.java
new file mode 100644
index 0000000..7dc6798
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/GarbageCollectorElapsedTimeMonitor.java
@@ -0,0 +1,44 @@
+/*
+ * 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.jmx.monitors;
+
+import org.apache.phoenix.pherf.jmx.Stat;
+
+import java.lang.management.GarbageCollectorMXBean;
+import java.lang.management.ManagementFactory;
+import java.util.List;
+
+public class GarbageCollectorElapsedTimeMonitor implements Monitor {
+
+    @Override
+    public Stat getStat() {
+        List<GarbageCollectorMXBean> beans = ManagementFactory.getGarbageCollectorMXBeans();
+        long average = 0;
+        Stat<Long> stat = null;
+        if (beans.size() > 0) {
+            for (GarbageCollectorMXBean bean : beans) {
+                average += bean.getCollectionTime();
+            }
+            stat = new Stat(average / beans.size());
+        } else {
+            stat = new Stat(0);
+        }
+        return stat;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/HeapMemoryMonitor.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/HeapMemoryMonitor.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/HeapMemoryMonitor.java
new file mode 100644
index 0000000..41f4746
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/HeapMemoryMonitor.java
@@ -0,0 +1,32 @@
+/*
+ * 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.jmx.monitors;
+
+import org.apache.phoenix.pherf.jmx.Stat;
+
+import java.lang.management.ManagementFactory;
+
+public class HeapMemoryMonitor implements Monitor {
+
+    @Override
+    public Stat getStat() {
+        Stat<Long> stat = new Stat(ManagementFactory.getMemoryMXBean().getHeapMemoryUsage().getUsed());
+        return stat;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/MaxMemoryMonitor.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/MaxMemoryMonitor.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/MaxMemoryMonitor.java
new file mode 100644
index 0000000..d53e552
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/MaxMemoryMonitor.java
@@ -0,0 +1,30 @@
+/*
+ * 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.jmx.monitors;
+
+import org.apache.phoenix.pherf.jmx.Stat;
+
+public class MaxMemoryMonitor implements Monitor {
+
+    @Override
+    public Stat getStat() {
+        Stat<Long> stat = new Stat(new Long(Runtime.getRuntime().maxMemory()));
+        return stat;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/Monitor.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/Monitor.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/Monitor.java
new file mode 100644
index 0000000..d856398
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/Monitor.java
@@ -0,0 +1,30 @@
+/*
+ * 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.jmx.monitors;
+
+import org.apache.phoenix.pherf.jmx.Stat;
+
+import javax.management.MXBean;
+
+@MXBean
+public interface Monitor {
+
+    public Stat getStat();
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/NonHeapMemoryMonitor.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/NonHeapMemoryMonitor.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/NonHeapMemoryMonitor.java
new file mode 100644
index 0000000..4f0a67b
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/NonHeapMemoryMonitor.java
@@ -0,0 +1,32 @@
+/*
+ * 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.jmx.monitors;
+
+import org.apache.phoenix.pherf.jmx.Stat;
+
+import java.lang.management.ManagementFactory;
+
+public class NonHeapMemoryMonitor implements Monitor {
+
+    @Override
+    public Stat getStat() {
+        Stat<Long> stat = new Stat(ManagementFactory.getMemoryMXBean().getNonHeapMemoryUsage().getUsed());
+        return stat;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/ObjectPendingFinalizationCountMonitor.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/ObjectPendingFinalizationCountMonitor.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/ObjectPendingFinalizationCountMonitor.java
new file mode 100644
index 0000000..254bf8c
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/ObjectPendingFinalizationCountMonitor.java
@@ -0,0 +1,33 @@
+/*
+ * 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.jmx.monitors;
+
+import org.apache.phoenix.pherf.jmx.Stat;
+
+import java.lang.management.ManagementFactory;
+import java.lang.management.MemoryMXBean;
+
+public class ObjectPendingFinalizationCountMonitor implements Monitor {
+
+    @Override
+    public Stat getStat() {
+        Stat<MemoryMXBean> stat = new Stat(ManagementFactory.getMemoryMXBean().getObjectPendingFinalizationCount());
+        return stat;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/ThreadMonitor.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/ThreadMonitor.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/ThreadMonitor.java
new file mode 100644
index 0000000..260af71
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/ThreadMonitor.java
@@ -0,0 +1,32 @@
+/*
+ * 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.jmx.monitors;
+
+import org.apache.phoenix.pherf.jmx.Stat;
+
+import java.lang.management.ManagementFactory;
+
+public class ThreadMonitor implements Monitor {
+
+    @Override
+    public Stat getStat() {
+        Stat<Integer> stat = new Stat(new Integer(ManagementFactory.getThreadMXBean().getThreadCount()));
+        return stat;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/TotalMemoryMonitor.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/TotalMemoryMonitor.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/TotalMemoryMonitor.java
new file mode 100644
index 0000000..6d7336a
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/jmx/monitors/TotalMemoryMonitor.java
@@ -0,0 +1,30 @@
+/*
+ * 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.jmx.monitors;
+
+import org.apache.phoenix.pherf.jmx.Stat;
+
+public class TotalMemoryMonitor implements Monitor {
+
+    @Override
+    public Stat getStat() {
+        Stat<Long> stat = new Stat(new Long(Runtime.getRuntime().totalMemory()));
+        return stat;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/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
new file mode 100644
index 0000000..8ddce34
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/loaddata/DataLoader.java
@@ -0,0 +1,365 @@
+/*
+ * 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.loaddata;
+
+import java.math.BigDecimal;
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.sql.Types;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.apache.phoenix.pherf.result.ResultUtil;
+import org.apache.phoenix.pherf.util.ResourceList;
+import org.apache.phoenix.pherf.util.RowCalculator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.phoenix.pherf.PherfConstants;
+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.exception.PherfException;
+import org.apache.phoenix.pherf.result.DataLoadThreadTime;
+import org.apache.phoenix.pherf.result.DataLoadTimeSummary;
+import org.apache.phoenix.pherf.rules.DataValue;
+import org.apache.phoenix.pherf.rules.RulesApplier;
+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 XMLConfigParser parser;
+    private final RulesApplier rulesApplier;
+    private final ResultUtil resultUtil;
+    private final ExecutorService pool;
+    private final Properties properties;
+
+    private final int threadPoolSize;
+    private final int batchSize;
+
+    public DataLoader(XMLConfigParser parser) throws Exception {
+        this(new ResourceList().getProperties(), parser);
+    }
+
+    /**
+     * Default the writers to use up all available cores for threads.
+     *
+     * @param parser
+     * @throws Exception
+     */
+    public DataLoader(Properties properties, XMLConfigParser parser) throws Exception {
+        this.parser = parser;
+        this.properties = properties;
+        this.rulesApplier = new RulesApplier(this.parser);
+        this.resultUtil = new ResultUtil();
+        int size = Integer.parseInt(properties.getProperty("pherf.default.dataloader.threadpool"));
+        this.threadPoolSize = (size == 0) ? Runtime.getRuntime().availableProcessors() : size;
+        this.pool = Executors.newFixedThreadPool(this.threadPoolSize);
+        String bSize = properties.getProperty("pherf.default.dataloader.batchsize");
+        this.batchSize = (bSize == null) ? PherfConstants.DEFAULT_BATCH_SIZE : Integer.parseInt(bSize);
+    }
+
+    public void execute() throws Exception {
+        try {
+            DataModel model = getParser().getDataModels().get(0);
+            DataLoadTimeSummary dataLoadTimeSummary = new DataLoadTimeSummary();
+            DataLoadThreadTime dataLoadThreadTime = new DataLoadThreadTime();
+
+            for (Scenario scenario : getParser().getScenarios()) {
+                List<Future> writeBatches = new ArrayList<Future>();
+                logger.info("\nLoading " + scenario.getRowCount()
+                        + " rows for " + scenario.getTableName());
+                long start = System.currentTimeMillis();
+
+                RowCalculator rowCalculator = new RowCalculator(getThreadPoolSize(), scenario.getRowCount());
+                for (int i = 0; i < getThreadPoolSize(); i++) {
+                    List<Column> phxMetaCols = pUtil.getColumnsFromPhoenix(
+                            scenario.getSchemaName(),
+                            scenario.getTableNameWithoutSchemaName(),
+                            pUtil.getConnection());
+                    int threadRowCount = rowCalculator.getNext();
+                    logger.info("Kick off thread (#" + i + ")for upsert with (" + threadRowCount + ") rows.");
+                    Future<Info> write = upsertData(scenario, phxMetaCols,
+                            scenario.getTableName(), threadRowCount, dataLoadThreadTime);
+                    writeBatches.add(write);
+                }
+
+                if (writeBatches.isEmpty()) {
+                    throw new PherfException(
+                            "Holy shit snacks! Throwing up hands in disbelief and exiting. Could not write data for some unknown reason.");
+                }
+
+                int sumRows = 0, sumDuration = 0;
+                // Wait for all the batch threads to complete
+                for (Future<Info> write : writeBatches) {
+                    Info writeInfo = write.get();
+                    sumRows += writeInfo.getRowCount();
+                    sumDuration += writeInfo.getDuration();
+                    logger.info("Executor writes complete with row count ("
+                                    + writeInfo.getRowCount()
+                                    + ") in Ms ("
+                                    + writeInfo.getDuration() + ")");
+                }
+                logger.info("Writes completed with total row count (" + sumRows
+                        + ") with total time of(" + sumDuration + ") Ms");
+                dataLoadTimeSummary.add(scenario.getTableName(), sumRows, (int) (System.currentTimeMillis() - start));
+
+
+                // always update stats for Phoenix base tables
+                updatePhoenixStats(scenario.getTableName());
+            }
+            resultUtil.write(dataLoadTimeSummary);
+            resultUtil.write(dataLoadThreadTime);
+
+        } finally {
+            pool.shutdown();
+        }
+    }
+
+    /**
+     * TODO Move this method to PhoenixUtil
+     * Update Phoenix table stats
+     *
+     * @param tableName
+     * @throws Exception
+     */
+    public void updatePhoenixStats(String tableName) throws Exception {
+        logger.info("Updating stats for " + tableName);
+        pUtil.executeStatement("UPDATE STATISTICS " + tableName);
+    }
+
+    public void printTableColumns(Scenario scenario) throws Exception {
+        Connection connection = null;
+        try {
+            connection = pUtil.getConnection();
+            List<Column> columnList = pUtil.getColumnsFromPhoenix(
+                    scenario.getSchemaName(),
+                    scenario.getTableNameWithoutSchemaName(), connection);
+
+            logger.debug("\n\nColumns from metadata:");
+            for (Column column : columnList) {
+                logger.debug("\nColumn name: [" + column.getName()
+                        + "]; type: [" + column.getType() + "]; length: ["
+                        + column.getLength() + "]");
+            }
+
+            if (null != scenario.getDataOverride()) {
+                logger.debug("\n\nColumns from override:");
+                for (Column column : scenario.getDataOverride().getColumn()) {
+                    logger.debug("\nColumn name: [" + column.getName() + "]; DataSequence: [" + column.getDataSequence()
+                            + "]; length: [" + column.getLength() + "]");
+                }
+            }
+
+        } finally {
+            if (connection != null) {
+                try {
+                    connection.close();
+                } catch (SQLException e) {
+                    // Swallow since we are closing anyway
+                    e.printStackTrace();
+                }
+            }
+        }
+    }
+
+    public Future<Info> upsertData(final Scenario scenario,
+                                   final List<Column> columns, final String tableName,
+                                   final int rowCount, final DataLoadThreadTime dataLoadThreadTime) {
+        Future<Info> future = pool.submit(new Callable<Info>() {
+            @Override
+            public Info call() throws Exception {
+                int rowsCreated = 0;
+                Info info = null;
+                long start = 0, duration = 0, totalDuration = 0;
+                SimpleDateFormat simpleDateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+                Connection connection = null;
+                try {
+                    connection = pUtil.getConnection();
+                    long logStartTime = System.currentTimeMillis();
+                    for (int i = 0; i < rowCount; i++) {
+                        String sql = buildSql(columns, tableName);
+                        PreparedStatement stmt = connection
+                                .prepareStatement(sql);
+                        stmt = buildStatement(scenario, columns, stmt, simpleDateFormat);
+                        start = System.currentTimeMillis();
+                        rowsCreated += stmt.executeUpdate();
+                        stmt.close();
+                        if ((i % getBatchSize()) == 0) {
+                            connection.commit();
+                            duration = System.currentTimeMillis() - start;
+                            logger.info("Committed Batch. Total " + tableName + " rows for this thread (" + this.hashCode() + ") in ("
+                                    + duration + ") Ms");
+
+                            if (i % PherfConstants.LOG_PER_NROWS == 0 && i != 0) {
+                                dataLoadThreadTime.add(tableName, Thread.currentThread().getName(), i, System.currentTimeMillis() - logStartTime);
+                                logStartTime = System.currentTimeMillis();
+                            }
+                        }
+                    }
+                } finally {
+                    if (connection != null) {
+                        try {
+                            connection.commit();
+                            duration = System.currentTimeMillis() - start;
+                            logger.info("Committed Final Batch. Duration (" + duration + ") Ms");
+                            connection.close();
+                        } catch (SQLException e) {
+                            // Swallow since we are closing anyway
+                            e.printStackTrace();
+                        }
+                    }
+                }
+                totalDuration = System.currentTimeMillis() - start;
+                return new Info(totalDuration, rowsCreated);
+            }
+        });
+        return future;
+    }
+
+    private PreparedStatement buildStatement(Scenario scenario,
+                                             List<Column> columns, PreparedStatement statement, SimpleDateFormat simpleDateFormat) throws Exception {
+        int count = 1;
+        for (Column column : columns) {
+
+            DataValue dataValue = getRulesApplier().getDataForRule(scenario,
+                    column);
+            switch (column.getType()) {
+                case VARCHAR:
+                    if (dataValue.getValue().equals("")) {
+                        statement.setNull(count, Types.VARCHAR);
+                    } else {
+                        statement.setString(count, dataValue.getValue());
+                    }
+                    break;
+                case CHAR:
+                    if (dataValue.getValue().equals("")) {
+                        statement.setNull(count, Types.CHAR);
+                    } else {
+                        statement.setString(count, dataValue.getValue());
+                    }
+                    break;
+                case DECIMAL:
+                    if (dataValue.getValue().equals("")) {
+                        statement.setNull(count, Types.DECIMAL);
+                    } else {
+                        statement.setBigDecimal(count,
+                                new BigDecimal(dataValue.getValue()));
+                    }
+                    break;
+                case INTEGER:
+                    if (dataValue.getValue().equals("")) {
+                        statement.setNull(count, Types.INTEGER);
+                    } else {
+                        statement.setInt(count,
+                                Integer.parseInt(dataValue.getValue()));
+                    }
+                    break;
+                case DATE:
+                    if (dataValue.getValue().equals("")) {
+                        statement.setNull(count, Types.DATE);
+                    } else {
+                        Date date = new java.sql.Date(simpleDateFormat.parse(dataValue.getValue()).getTime());
+                        statement.setDate(count, date);
+                    }
+                    break;
+                default:
+                    break;
+            }
+            count++;
+        }
+        return statement;
+    }
+
+    private String buildSql(final List<Column> columns, final String tableName) {
+        StringBuilder builder = new StringBuilder();
+        builder.append("upsert into ");
+        builder.append(tableName);
+        builder.append(" (");
+        int count = 1;
+        for (Column column : columns) {
+            builder.append(column.getName());
+            if (count < columns.size()) {
+                builder.append(",");
+            } else {
+                builder.append(")");
+            }
+            count++;
+        }
+        builder.append(" VALUES (");
+        for (int i = 0; i < columns.size(); i++) {
+            if (i < columns.size() - 1) {
+                builder.append("?,");
+            } else {
+                builder.append("?)");
+            }
+        }
+        return builder.toString();
+    }
+
+    public XMLConfigParser getParser() {
+        return parser;
+    }
+
+    public RulesApplier getRulesApplier() {
+        return rulesApplier;
+    }
+
+    public int getBatchSize() {
+        return batchSize;
+    }
+
+    public int getThreadPoolSize() {
+        return threadPoolSize;
+    }
+
+    private class Info {
+
+        private final int rowCount;
+        private final long duration;
+
+        public Info(long duration, int rows) {
+            this(0, 0, 0, duration, rows);
+        }
+
+        public Info(int regionSize, int completedIterations, int timesSeen,
+                    long duration, int rows) {
+            this.duration = duration;
+            this.rowCount = rows;
+        }
+
+        public long getDuration() {
+            return duration;
+        }
+
+        public int getRowCount() {
+            return rowCount;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/DataLoadThreadTime.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/DataLoadThreadTime.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/DataLoadThreadTime.java
new file mode 100644
index 0000000..47aa2bc
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/DataLoadThreadTime.java
@@ -0,0 +1,95 @@
+/*
+ * 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.result;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.phoenix.pherf.PherfConstants;
+
+public class DataLoadThreadTime {
+	private List<WriteThreadTime> threadTime = new ArrayList<WriteThreadTime>();
+
+	public List<WriteThreadTime> getThreadTime() {
+		return threadTime;
+	}
+
+	public void setThreadTime(List<WriteThreadTime> threadTime) {
+		this.threadTime = threadTime;
+	}
+	
+	public void add(String tableName, String threadName, int rowsUpserted, long timeInMsPerMillionRows) {
+		threadTime.add(new WriteThreadTime(tableName, threadName, rowsUpserted, timeInMsPerMillionRows));	
+	}
+	
+	public String getCsvTitle() {
+		return "TABLE_NAME,THREAD_NAME,ROWS_UPSERTED,TIME_IN_MS_PER_" + PherfConstants.LOG_PER_NROWS + "_ROWS\n";
+	}
+}
+
+class WriteThreadTime {
+	private String tableName;
+	private String threadName;
+	private int rowsUpserted;
+	private long timeInMsPerMillionRows;
+	
+	public WriteThreadTime(String tableName, String threadName, int rowsUpserted, long timeInMsPerMillionRows) {
+		this.tableName = tableName;
+		this.threadName = threadName;
+		this.rowsUpserted = rowsUpserted;
+		this.timeInMsPerMillionRows = timeInMsPerMillionRows;
+	}
+	
+	public String getTableName() {
+		return tableName;
+	}
+	public void setTableName(String tableName) {
+		this.tableName = tableName;
+	}
+	public String getThreadName() {
+		return threadName;
+	}
+	public void setThreadName(String threadName) {
+		this.threadName = threadName;
+	}
+	public long getTimeInMsPerMillionRows() {
+		return timeInMsPerMillionRows;
+	}
+	public void setTimeInMsPerMillionRows(long timeInMsPerMillionRows) {
+		this.timeInMsPerMillionRows = timeInMsPerMillionRows;
+	}
+	
+	public List<ResultValue> getCsvRepresentation(ResultUtil util) {
+        List<ResultValue> rowValues = new ArrayList<>();
+        rowValues.add(new ResultValue(util.convertNull(getTableName())));
+        rowValues.add(new ResultValue(util.convertNull(getThreadName())));
+        rowValues.add(new ResultValue(util.convertNull(String.valueOf(getRowsUpserted()))));
+        rowValues.add(new ResultValue(util.convertNull(String.valueOf(getTimeInMsPerMillionRows()))));
+
+        return rowValues;
+	}
+
+	public int getRowsUpserted() {
+		return rowsUpserted;
+	}
+
+	public void setRowsUpserted(int rowsUpserted) {
+		this.rowsUpserted = rowsUpserted;
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/DataLoadTimeSummary.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/DataLoadTimeSummary.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/DataLoadTimeSummary.java
new file mode 100644
index 0000000..ae4838b
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/DataLoadTimeSummary.java
@@ -0,0 +1,84 @@
+/*
+ * 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.result;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class DataLoadTimeSummary {
+	private List<TableLoadTime> tableLoadTime = new ArrayList<TableLoadTime>();
+
+	public List<TableLoadTime> getTableLoadTime() {
+		return tableLoadTime;
+	}
+	
+	public void add(String tableName, int rowCount, int durationInMs) {
+		tableLoadTime.add(new TableLoadTime(tableName, rowCount, durationInMs));
+	}
+
+	public void setTableLoadTime(List<TableLoadTime> tableLoadTime) {
+		this.tableLoadTime = tableLoadTime;
+	}
+
+}
+
+class TableLoadTime {
+	private int durationInMs;
+	private String tableName;
+	private int rowCount;
+
+	public TableLoadTime(String tableName, int rowCount, int durationInMs) {
+		this.tableName = tableName;
+		this.rowCount = rowCount;
+		this.durationInMs = durationInMs;
+	}
+	
+	public List<ResultValue> getCsvRepresentation(ResultUtil util) {
+        List<ResultValue> rowValues = new ArrayList<>();
+        rowValues.add(new ResultValue(util.convertNull(getTableName())));
+        rowValues.add(new ResultValue(util.convertNull(String.valueOf(getRowCount()))));
+        rowValues.add(new ResultValue(util.convertNull(String.valueOf(getDurationInMs()))));
+
+        return rowValues;
+    }
+
+	public int getDurationInMs() {
+		return durationInMs;
+	}
+
+	public void setDurationInMs(int durationInMs) {
+		this.durationInMs = durationInMs;
+	}
+
+	public String getTableName() {
+		return tableName;
+	}
+
+	public void setTableName(String tableName) {
+		this.tableName = tableName;
+	}
+
+	public int getRowCount() {
+		return rowCount;
+	}
+
+	public void setRowCount(int rowCount) {
+		this.rowCount = rowCount;
+	}
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/DataModelResult.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/DataModelResult.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/DataModelResult.java
new file mode 100644
index 0000000..71ffeaf
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/DataModelResult.java
@@ -0,0 +1,77 @@
+/*
+ * 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.result;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import javax.xml.bind.annotation.XmlAttribute;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.phoenix.pherf.configuration.DataModel;
+
+@XmlRootElement(namespace = "org.apache.phoenix.pherf.result")
+public class DataModelResult extends DataModel {
+	private List<ScenarioResult> scenarioResult = new ArrayList<ScenarioResult>();
+	private String zookeeper;
+
+	public List<ScenarioResult> getScenarioResult() {
+		return scenarioResult;
+	}
+
+	public void setScenarioResult(List<ScenarioResult> scenarioResult) {
+		this.scenarioResult = scenarioResult;
+	}
+	
+	public DataModelResult() {
+	}
+
+    private DataModelResult(String name, String release, String zookeeper) {
+        this.setName(name);
+        this.setRelease(release);
+        this.zookeeper = zookeeper;
+    }
+
+    /**
+     * Copy constructor
+     * 
+     * @param dataModelResult
+     */
+    public DataModelResult(DataModelResult dataModelResult) {
+        this(dataModelResult.getName(), dataModelResult.getRelease(), dataModelResult.getZookeeper());
+        this.scenarioResult = dataModelResult.getScenarioResult();
+    }
+	
+	public DataModelResult(DataModel dataModel, String zookeeper) {
+	    this(dataModel.getName(), dataModel.getRelease(), zookeeper);
+	}
+	
+	public DataModelResult(DataModel dataModel) {
+		this(dataModel, null);
+	}
+
+	@XmlAttribute()
+	public String getZookeeper() {
+		return zookeeper;
+	}
+
+	public void setZookeeper(String zookeeper) {
+		this.zookeeper = zookeeper;
+	}
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/QueryResult.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/QueryResult.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/QueryResult.java
new file mode 100644
index 0000000..ac50301
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/QueryResult.java
@@ -0,0 +1,141 @@
+/*
+ * 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.result;
+
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+
+import org.apache.phoenix.pherf.PherfConstants.RunMode;
+import org.apache.phoenix.pherf.configuration.Query;
+import org.apache.phoenix.util.DateUtil;
+
+public class QueryResult extends Query {
+    private List<ThreadTime> threadTimes = new ArrayList<ThreadTime>();
+
+    public synchronized List<ThreadTime> getThreadTimes() {
+        return this.threadTimes;
+    }
+
+    public synchronized void setThreadTimes(List<ThreadTime> threadTimes) {
+        this.threadTimes = threadTimes;
+    }
+
+    public QueryResult(Query query) {
+        this.setStatement(query.getStatement());
+        this.setExpectedAggregateRowCount(query.getExpectedAggregateRowCount());
+        this.setTenantId(query.getTenantId());
+        this.setDdl(query.getDdl());
+        this.setQueryGroup(query.getQueryGroup());
+        this.setId(query.getId());
+    }
+
+    public QueryResult() {
+    }
+
+    public Date getStartTime() {
+        Date startTime = null;
+        for (ThreadTime tt : getThreadTimes()) {
+            Date currStartTime = tt.getStartTime();
+            if (null != currStartTime) {
+                if (null == startTime) {
+                    startTime = currStartTime;
+                } else if (currStartTime.compareTo(startTime) < 0) {
+                    startTime = currStartTime;
+                }
+            }
+        }
+        return startTime;
+    }
+
+    public int getAvgMaxRunTimeInMs() {
+        int totalRunTime = 0;
+        for (ThreadTime tt : getThreadTimes()) {
+            if (null != tt.getMaxTimeInMs()) {
+                totalRunTime += tt.getMaxTimeInMs().getElapsedDurationInMs();
+            }
+        }
+        return totalRunTime / getThreadTimes().size();
+    }
+
+    public int getAvgMinRunTimeInMs() {
+        int totalRunTime = 0;
+        for (ThreadTime tt : getThreadTimes()) {
+            if (null != tt.getMinTimeInMs()) {
+                totalRunTime += tt.getMinTimeInMs().getElapsedDurationInMs();
+            }
+        }
+        return totalRunTime / getThreadTimes().size();
+    }
+
+    public int getAvgRunTimeInMs() {
+        int totalRunTime = 0;
+        for (ThreadTime tt : getThreadTimes()) {
+            if (null != tt.getAvgTimeInMs()) {
+                totalRunTime += tt.getAvgTimeInMs();
+            }
+        }
+        return totalRunTime / getThreadTimes().size();
+    }
+
+    public List<ResultValue> getCsvRepresentation(ResultUtil util) {
+        List<ResultValue> rowValues = new ArrayList<>();
+        rowValues.add(new ResultValue(util.convertNull(getStartTimeText())));
+        rowValues.add(new ResultValue(util.convertNull(this.getQueryGroup())));
+        rowValues.add(new ResultValue(util.convertNull(this.getStatement())));
+        rowValues.add(new ResultValue(util.convertNull(this.getTenantId())));
+        rowValues.add(new ResultValue(util.convertNull(String.valueOf(getAvgMaxRunTimeInMs()))));
+        rowValues.add(new ResultValue(util.convertNull(String.valueOf(getAvgRunTimeInMs()))));
+        rowValues.add(new ResultValue(util.convertNull(String.valueOf(getAvgMinRunTimeInMs()))));
+        rowValues.add(new ResultValue(util.convertNull(String.valueOf(getRunCount()))));
+        return rowValues;
+    }
+
+    private int getRunCount() {
+        int totalRunCount = 0;
+        for (ThreadTime tt : getThreadTimes()) {
+            totalRunCount += tt.getRunCount();
+        }
+        return totalRunCount;
+    }
+
+    public List<List<ResultValue>> getCsvDetailedRepresentation(ResultUtil util, RunMode runMode) {
+        List<List<ResultValue>> rows = new ArrayList<>();
+        for (ThreadTime tt : getThreadTimes()) {
+            for (List<ResultValue> runTime : runMode == RunMode.PERFORMANCE ?
+                    tt.getCsvPerformanceRepresentation(util) :
+                    tt.getCsvFunctionalRepresentation(util)) {
+                List<ResultValue> rowValues = new ArrayList<>();
+                rowValues.add(new ResultValue(util.convertNull(getStartTimeText())));
+                rowValues.add(new ResultValue(util.convertNull(this.getQueryGroup())));
+                rowValues.add(new ResultValue(util.convertNull(this.getStatement())));
+                rowValues.add(new ResultValue(util.convertNull(this.getTenantId())));
+                rowValues.addAll(runTime);
+                rows.add(rowValues);
+            }
+        }
+        return rows;
+    }
+
+    private String getStartTimeText() {
+        return (null == this.getStartTime())
+                ? ""
+                : DateUtil.DEFAULT_MS_DATE_FORMATTER.format(this.getStartTime());
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/QuerySetResult.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/QuerySetResult.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/QuerySetResult.java
new file mode 100644
index 0000000..c76c2e5
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/QuerySetResult.java
@@ -0,0 +1,47 @@
+/*
+ * 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.result;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.phoenix.pherf.configuration.QuerySet;
+
+public class QuerySetResult extends QuerySet {
+	
+	private List<QueryResult> queryResults = new ArrayList<QueryResult>();
+
+	public QuerySetResult(QuerySet querySet) {
+		this.setConcurrency(querySet.getConcurrency());
+		this.setNumberOfExecutions(querySet.getNumberOfExecutions());
+		this.setExecutionDurationInMs(querySet.getExecutionDurationInMs());
+		this.setExecutionType(querySet.getExecutionType());
+	}
+	
+	public QuerySetResult() {
+	}
+
+	public List<QueryResult> getQueryResults() {
+		return queryResults;
+	}
+
+	public void setQueryResults(List<QueryResult> queryResults) {
+		this.queryResults = queryResults;
+	}	
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/Result.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/Result.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/Result.java
new file mode 100644
index 0000000..104e388
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/Result.java
@@ -0,0 +1,54 @@
+/*
+ * 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.result;
+
+import org.apache.phoenix.pherf.result.file.ResultFileDetails;
+
+import java.util.List;
+
+/**
+ * Common container for Pherf results.
+ */
+public class Result {
+    private final List<ResultValue> resultValues;
+    private final ResultFileDetails type;
+    private final String header;
+
+    /**
+     *
+     * @param type {@link org.apache.phoenix.pherf.result.file.ResultFileDetails} Currently unused, but gives metadata about the
+     *                                                           contents of the result.
+     * @param header Used for CSV, otherwise pass null. For CSV pass comma separated string of header fields.
+     * @param messageValues List<{@link ResultValue} All fields combined represent the data
+     *                      for a row to be written.
+     */
+    public Result(ResultFileDetails type, String header, List<ResultValue> messageValues) {
+        this.resultValues = messageValues;
+        this.header = header;
+        this.type = type;
+    }
+
+    public List<ResultValue> getResultValues() {
+        return resultValues;
+    }
+
+    public String getHeader() {
+        return header;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/ResultHandler.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/ResultHandler.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/ResultHandler.java
new file mode 100644
index 0000000..f650cbb
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/ResultHandler.java
@@ -0,0 +1,37 @@
+/*
+ * 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.result;
+
+import org.apache.phoenix.pherf.result.file.ResultFileDetails;
+
+import java.util.List;
+
+/**
+ * This is a common interface for working with Pherf results in various output formats. Implementations of this
+ * interface can deal with particular details for that format while giving callers to output a simple API to report
+ * against.
+ */
+public interface ResultHandler {
+    public void write(Result result) throws Exception;
+    public void flush() throws Exception;
+    public void close() throws Exception;
+    public List<Result> read() throws Exception;
+    public boolean isClosed();
+    public ResultFileDetails getResultFileDetails();
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/ResultManager.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/ResultManager.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/ResultManager.java
new file mode 100644
index 0000000..523feb4
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/ResultManager.java
@@ -0,0 +1,103 @@
+/*
+ * 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.result;
+
+import org.apache.phoenix.pherf.PherfConstants;
+import org.apache.phoenix.pherf.PherfConstants.RunMode;
+import org.apache.phoenix.pherf.result.file.ResultFileDetails;
+import org.apache.phoenix.pherf.result.impl.CSVResultHandler;
+import org.apache.phoenix.pherf.result.impl.ImageResultHandler;
+import org.apache.phoenix.pherf.result.impl.XMLResultHandler;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class ResultManager {
+    private final List<ResultHandler> resultHandlers;
+    private final ResultUtil util;
+    private final PherfConstants.RunMode runMode;
+
+
+    public ResultManager(String fileNameSeed, PherfConstants.RunMode runMode) {
+        this(runMode, Arrays.asList(
+                new XMLResultHandler(fileNameSeed, ResultFileDetails.XML),
+                new ImageResultHandler(fileNameSeed, ResultFileDetails.IMAGE),
+						new CSVResultHandler(
+								fileNameSeed,
+								runMode == RunMode.PERFORMANCE ? ResultFileDetails.CSV_DETAILED_PERFORMANCE
+										: ResultFileDetails.CSV_DETAILED_FUNCTIONAL),
+                new CSVResultHandler(fileNameSeed, ResultFileDetails.CSV_AGGREGATE_PERFORMANCE)
+        ));
+    }
+
+    public ResultManager(PherfConstants.RunMode runMode, List<ResultHandler> resultHandlers) {
+        this.resultHandlers = resultHandlers;
+        util = new ResultUtil();
+        this.runMode = runMode;
+    }
+
+    /**
+     * Write out the result to each writer in the pool
+     *
+     * @param result {@link DataModelResult}
+     * @throws Exception
+     */
+    public synchronized void write(DataModelResult result) throws Exception {
+        try {
+            util.ensureBaseResultDirExists();
+            final DataModelResult dataModelResultCopy = new DataModelResult(result);
+            for (ResultHandler handler : resultHandlers) {
+                util.write(handler, dataModelResultCopy, runMode);
+            }
+        } finally {
+            for (ResultHandler handler : resultHandlers) {
+                try {
+                    if (handler != null) {
+                        handler.flush();
+                        handler.close();
+                    }
+                } catch (Exception e) {
+                    e.printStackTrace();
+                }
+            }
+        }
+    }
+
+    /**
+     * Write a combined set of results for each result in the list.
+     * @param dataModelResults List<{@link DataModelResult > </>}
+     * @throws Exception
+     */
+    public synchronized void write(List<DataModelResult> dataModelResults) throws Exception {
+        util.ensureBaseResultDirExists();
+
+        CSVResultHandler detailsCSVWriter = null;
+        try {
+            detailsCSVWriter = new CSVResultHandler(PherfConstants.COMBINED_FILE_NAME, ResultFileDetails.CSV_DETAILED_PERFORMANCE);
+            for (DataModelResult dataModelResult : dataModelResults) {
+                util.write(detailsCSVWriter, dataModelResult, runMode);
+            }
+        } finally {
+            if (detailsCSVWriter != null) {
+                detailsCSVWriter.flush();
+                detailsCSVWriter.close();
+            }
+        }
+    }
+}


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

Posted by mu...@apache.org.
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/36b88651
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/36b88651
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/36b88651

Branch: refs/heads/master
Commit: 36b88651cbeb680a7d06828bcb5d61d440341ad2
Parents: 94ade6a
Author: Mujtaba <mu...@apache.org>
Authored: Fri Mar 13 10:45:19 2015 -0700
Committer: Mujtaba <mu...@apache.org>
Committed: Fri Mar 13 10:45:19 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/36b88651/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/36b88651/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/36b88651/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/36b88651/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/36b88651/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/36b88651/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/36b88651/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/36b88651/phoenix-pherf/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-pherf/pom.xml b/phoenix-pherf/pom.xml
new file mode 100644
index 0000000..a5e445a
--- /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>5.0.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>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/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/36b88651/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/36b88651/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/36b88651/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/36b88651/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/36b88651/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/36b88651/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/36b88651/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/36b88651/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/36b88651/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/36b88651/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/36b88651/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/36b88651/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;
+    }
+
+}


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

Posted by mu...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/ResultUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/ResultUtil.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/ResultUtil.java
new file mode 100644
index 0000000..a2660a0
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/ResultUtil.java
@@ -0,0 +1,271 @@
+/*
+ * 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.result;
+
+import org.apache.phoenix.pherf.PherfConstants;
+import org.apache.phoenix.pherf.PherfConstants.RunMode;
+import org.apache.phoenix.pherf.result.file.ResultFileDetails;
+import org.apache.phoenix.pherf.result.impl.CSVResultHandler;
+import org.apache.phoenix.pherf.result.impl.ImageResultHandler;
+import org.apache.phoenix.pherf.result.impl.XMLResultHandler;
+import org.apache.phoenix.pherf.util.PhoenixUtil;
+
+import java.io.*;
+import java.text.Format;
+import java.text.SimpleDateFormat;
+import java.util.*;
+import java.util.List;
+
+public class ResultUtil {
+
+    /*This variable needs to be static - Otherwise multiple result files will be generated*/
+    private static String FILE_SUFFIX = null;
+
+    /**
+     * Overload for write all results type to file
+     * <p/>
+     * TODO Remove when we are sure results are stable. Currently there are no more references to this.
+     *
+     * @param dataModelResult
+     * @param fileName
+     * @throws javax.xml.bind.JAXBException
+     * @throws IOException
+     */
+    public synchronized void writeResultToFile(DataModelResult dataModelResult, String fileName, RunMode runMode) throws Exception {
+
+        ResultHandler detailsCSVHandler;
+        ResultHandler aggregateCSVHandler;
+        ResultHandler xmlResultHandler;
+        ResultHandler imageResultHandler;
+        List<ResultHandler> handlers = new ArrayList<>();
+        try {
+            ensureBaseResultDirExists();
+            final DataModelResult dataModelResultCopy = new DataModelResult(dataModelResult);
+
+            detailsCSVHandler = new CSVResultHandler(fileName, ResultFileDetails.CSV_DETAILED_PERFORMANCE);
+            handlers.add(detailsCSVHandler);
+            xmlResultHandler = new XMLResultHandler(fileName, ResultFileDetails.XML);
+            handlers.add(xmlResultHandler);
+            aggregateCSVHandler = new CSVResultHandler(fileName, ResultFileDetails.CSV_AGGREGATE_PERFORMANCE);
+            handlers.add(aggregateCSVHandler);
+            imageResultHandler = new ImageResultHandler(fileName, ResultFileDetails.IMAGE);
+            handlers.add(imageResultHandler);
+
+            // XML results
+            write(xmlResultHandler, dataModelResultCopy, runMode);
+            // JPG result visualization
+            write(imageResultHandler, dataModelResultCopy, runMode);
+            // CSV results
+            write(aggregateCSVHandler, dataModelResultCopy, runMode);
+            // CSV results details
+            write(detailsCSVHandler, dataModelResultCopy, runMode);
+
+        } finally {
+            for (ResultHandler handler : handlers) {
+                try {
+                    if (handler != null) {
+                        handler.flush();
+                        handler.close();
+                    }
+                } catch (Exception e) {
+                    e.printStackTrace();
+                }
+            }
+        }
+    }
+
+    /**
+     * Write data load time details
+     *
+     * @param dataLoadThreadTime {@link DataLoadThreadTime}
+     * @throws IOException
+     */
+    public synchronized void write(DataLoadThreadTime dataLoadThreadTime) throws IOException {
+        ensureBaseResultDirExists();
+
+        CSVResultHandler writer = null;
+        try {
+            if (!dataLoadThreadTime.getThreadTime().isEmpty()) {
+                writer = new CSVResultHandler("Data_Load_Details", ResultFileDetails.CSV);
+                for (WriteThreadTime writeThreadTime : dataLoadThreadTime.getThreadTime()) {
+                    List<ResultValue> rowValues = new ArrayList<>();
+                    rowValues.add(new ResultValue(PhoenixUtil.getZookeeper()));
+                    rowValues.addAll(writeThreadTime.getCsvRepresentation(this));
+                    Result result = new Result(ResultFileDetails.CSV_DETAILED_PERFORMANCE, "ZK," + dataLoadThreadTime.getCsvTitle(), rowValues);
+                    writer.write(result);
+                }
+            }
+        } finally {
+            if (writer != null) {
+                writer.flush();
+                writer.close();
+            }
+        }
+    }
+
+    /**
+     * Write data load time summary
+     *
+     * @param dataLoadTime
+     * @throws IOException
+     */
+    public synchronized void write(DataLoadTimeSummary dataLoadTime) throws IOException {
+        ensureBaseResultDirExists();
+
+        CSVResultHandler writer = null;
+        ResultFileDetails resultFileDetails = ResultFileDetails.CSV_AGGREGATE_DATA_LOAD;
+        try {
+            writer = new CSVResultHandler("Data_Load_Summary", ResultFileDetails.CSV);
+            for (TableLoadTime loadTime : dataLoadTime.getTableLoadTime()) {
+                List<ResultValue> rowValues = new ArrayList<>();
+                rowValues.add(new ResultValue(PhoenixUtil.getZookeeper()));
+                rowValues.addAll(loadTime.getCsvRepresentation(this));
+                Result result = new Result(resultFileDetails, resultFileDetails.getHeader().toString(), rowValues);
+                writer.write(result);
+            }
+        } finally {
+            if (writer != null) {
+                writer.flush();
+                writer.close();
+            }
+        }
+    }
+
+    // TODO remove when stable. There are no more references to this method.
+    public synchronized void write(List<DataModelResult> dataModelResults, RunMode runMode) throws Exception {
+        ensureBaseResultDirExists();
+
+        CSVResultHandler detailsCSVWriter = null;
+        try {
+            detailsCSVWriter = new CSVResultHandler(PherfConstants.COMBINED_FILE_NAME, ResultFileDetails.CSV_DETAILED_PERFORMANCE);
+            for (DataModelResult dataModelResult : dataModelResults) {
+                write(detailsCSVWriter, dataModelResult, runMode);
+            }
+        } finally {
+            if (detailsCSVWriter != null) {
+                detailsCSVWriter.flush();
+                detailsCSVWriter.close();
+            }
+        }
+    }
+
+    public synchronized void write(ResultHandler resultHandler, DataModelResult dataModelResult, RunMode runMode) throws Exception {
+        ResultFileDetails resultFileDetails = resultHandler.getResultFileDetails();
+        switch (resultFileDetails) {
+            case CSV_AGGREGATE_PERFORMANCE:
+            case CSV_DETAILED_PERFORMANCE:
+            case CSV_DETAILED_FUNCTIONAL:
+                List<List<ResultValue>> rowDetails = getCSVResults(dataModelResult, resultFileDetails, runMode);
+                for (List<ResultValue> row : rowDetails) {
+                    Result result = new Result(resultFileDetails, resultFileDetails.getHeader().toString(), row);
+                    resultHandler.write(result);
+                }
+                break;
+            default:
+                List<ResultValue> resultValue = new ArrayList();
+                resultValue.add(new ResultValue<>(dataModelResult));
+                resultHandler.write(new Result(resultFileDetails, null, resultValue));
+                break;
+        }
+    }
+
+    public void ensureBaseResultDirExists() {
+        ensureBaseDirExists(PherfConstants.RESULT_DIR);
+    }
+
+    /**
+     * Utility method to check if base result dir exists
+     */
+    public void ensureBaseDirExists(String directory) {
+        File baseDir = new File(directory);
+        if (!baseDir.exists()) {
+            baseDir.mkdir();
+        }
+    }
+
+    public String getSuffix() {
+        if (null == FILE_SUFFIX) {
+            Date date = new Date();
+            Format formatter = new SimpleDateFormat("YYYY-MM-dd_hh-mm-ss");
+            FILE_SUFFIX = "_" + formatter.format(date);
+        }
+        return FILE_SUFFIX;
+    }
+
+    public String convertNull(String str) {
+        if ((str == null) || str.equals("")) {
+            return "null";
+        }
+        return str;
+    }
+
+    private List<List<ResultValue>> getCSVResults(DataModelResult dataModelResult, ResultFileDetails resultFileDetails, RunMode runMode) {
+        List<List<ResultValue>> rowList = new ArrayList<>();
+
+        for (ScenarioResult result : dataModelResult.getScenarioResult()) {
+            for (QuerySetResult querySetResult : result.getQuerySetResult()) {
+                for (QueryResult queryResult : querySetResult.getQueryResults()) {
+                    switch (resultFileDetails) {
+                        case CSV_AGGREGATE_PERFORMANCE:
+                            List<ResultValue> csvResult = queryResult.getCsvRepresentation(this);
+                            rowList.add(csvResult);
+                            break;
+                        case CSV_DETAILED_PERFORMANCE:
+                        case CSV_DETAILED_FUNCTIONAL:
+                            List<List<ResultValue>> detailedRows = queryResult.getCsvDetailedRepresentation(this, runMode);
+                            for (List<ResultValue> detailedRowList : detailedRows) {
+                                List<ResultValue> valueList = new ArrayList<>();
+                                valueList.add(new ResultValue(convertNull(result.getTableName())));
+                                valueList.add(new ResultValue(convertNull(result.getName())));
+                                valueList.add(new ResultValue(convertNull(dataModelResult.getZookeeper())));
+                                valueList.add(new ResultValue(convertNull(String.valueOf(result.getRowCount()))));
+                                valueList.add(new ResultValue(convertNull(String.valueOf(querySetResult.getNumberOfExecutions()))));
+                                valueList.add(new ResultValue(convertNull(String.valueOf(querySetResult.getExecutionType()))));
+                                if (result.getPhoenixProperties() != null) {
+                                    String props = buildProperty(result);
+                                    valueList.add(new ResultValue(convertNull(props)));
+                                } else {
+                                    valueList.add(new ResultValue("null"));
+                                }
+                                valueList.addAll(detailedRowList);
+                                rowList.add(valueList);
+                            }
+                            break;
+                        default:
+                            break;
+                    }
+                }
+            }
+        }
+        return rowList;
+    }
+
+    private String buildProperty(ScenarioResult result) {
+        StringBuffer sb = new StringBuffer();
+        boolean firstPartialSeparator = true;
+
+        for (Map.Entry<String, String> entry : result.getPhoenixProperties().entrySet()) {
+            if (!firstPartialSeparator)
+                sb.append("|");
+            firstPartialSeparator = false;
+            sb.append(entry.getKey() + "=" + entry.getValue());
+        }
+        return sb.toString();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/ResultValue.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/ResultValue.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/ResultValue.java
new file mode 100644
index 0000000..38abd65
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/ResultValue.java
@@ -0,0 +1,40 @@
+/*
+ * 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.result;
+
+/**
+ * Generic box container for a result value. This class allows for writing results of any type easily
+ * @param <T>
+ */
+public class ResultValue<T> {
+    T resultValue;
+
+    public ResultValue(T resultValue) {
+        this.resultValue = resultValue;
+    }
+
+    public T getResultValue() {
+        return resultValue;
+    }
+
+    @Override
+    public String toString() {
+        return resultValue.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/RunTime.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/RunTime.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/RunTime.java
new file mode 100644
index 0000000..3a9f40c
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/RunTime.java
@@ -0,0 +1,114 @@
+/*
+ * 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.result;
+
+import java.util.Comparator;
+import java.util.Date;
+
+import javax.xml.bind.annotation.XmlAttribute;
+
+public class RunTime implements Comparator<RunTime>, Comparable<RunTime> {
+	private Date startTime;
+	private Integer elapsedDurationInMs;
+	private String message;
+	private Long resultRowCount;
+	private String explainPlan;
+
+	public RunTime(Integer elapsedDurationInMs) {
+		this(null, elapsedDurationInMs);
+	}
+	
+	public RunTime(Long resultRowCount, Integer elapsedDurationInMs) {
+		this(null, resultRowCount, elapsedDurationInMs);
+	}
+	
+	public RunTime(Date startTime, Long resultRowCount, Integer elapsedDurationInMs) {
+		this(null, null, startTime, resultRowCount, elapsedDurationInMs);
+	}
+	
+	public RunTime(String message, Date startTime, Long resultRowCount, Integer elapsedDurationInMs) {
+		this(message, null, startTime, resultRowCount, elapsedDurationInMs);
+	}
+	
+	public RunTime(String message, String explainPlan, Date startTime, Long resultRowCount, Integer elapsedDurationInMs) {
+		this.elapsedDurationInMs = elapsedDurationInMs;
+		this.startTime = startTime;
+		this.resultRowCount = resultRowCount;
+		this.message = message;
+		this.explainPlan = explainPlan;
+	}
+	
+	public RunTime() {
+	}
+	
+	@XmlAttribute()
+	public Date getStartTime() {
+		return startTime;
+	}
+
+	public void setStartTime(Date startTime) {
+		this.startTime = startTime;
+	}
+	
+	@XmlAttribute()
+	public Integer getElapsedDurationInMs() {
+		return elapsedDurationInMs;
+	}
+
+	public void setElapsedDurationInMs(Integer elapsedDurationInMs) {
+		this.elapsedDurationInMs = elapsedDurationInMs;
+	}
+
+	@Override
+	public int compare(RunTime r1, RunTime r2) {
+		return r1.getElapsedDurationInMs().compareTo(r2.getElapsedDurationInMs());
+	}
+
+	@Override
+	public int compareTo(RunTime o) {
+		return compare(this, o);
+	}
+
+	@XmlAttribute()
+	public String getMessage() {
+		return message;
+	}
+
+	public void setMessage(String message) {
+		this.message = message;
+	}
+	
+	@XmlAttribute()
+	public String getExplainPlan() {
+		return explainPlan;
+	}
+
+	public void setExplainPlan(String explainPlan) {
+		this.explainPlan = explainPlan;
+	}
+
+	@XmlAttribute()
+	public Long getResultRowCount() {
+		return resultRowCount;
+	}
+
+	public void setResultRowCount(Long resultRowCount) {
+		this.resultRowCount = resultRowCount;
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/ScenarioResult.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/ScenarioResult.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/ScenarioResult.java
new file mode 100644
index 0000000..900860a
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/ScenarioResult.java
@@ -0,0 +1,47 @@
+/*
+ * 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.result;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.phoenix.pherf.configuration.Scenario;
+
+public class ScenarioResult extends Scenario {
+
+	private List<QuerySetResult> querySetResult = new ArrayList<QuerySetResult>();
+	
+	public List<QuerySetResult> getQuerySetResult() {
+		return querySetResult;
+	}
+
+	public void setQuerySetResult(List<QuerySetResult> querySetResult) {
+		this.querySetResult = querySetResult;
+	}
+	
+	public ScenarioResult() {
+	}
+	
+	public ScenarioResult(Scenario scenario) {
+		this.setDataOverride(scenario.getDataOverride());
+		this.setPhoenixProperties(scenario.getPhoenixProperties());
+		this.setRowCount(scenario.getRowCount());
+		this.setTableName(scenario.getTableName());
+		this.setName(scenario.getName());
+	}
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/ThreadTime.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/ThreadTime.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/ThreadTime.java
new file mode 100644
index 0000000..89e4aa6
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/ThreadTime.java
@@ -0,0 +1,141 @@
+/*
+ * 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.result;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Date;
+import java.util.List;
+
+import javax.xml.bind.annotation.XmlAttribute;
+
+public class ThreadTime {
+    private List<RunTime> runTimesInMs = Collections.synchronizedList(new ArrayList<RunTime>());
+    private String threadName;
+
+    public synchronized List<RunTime> getRunTimesInMs() {
+        return this.runTimesInMs;
+    }
+
+    public synchronized void setRunTimesInMs(List<RunTime> runTimesInMs) {
+        this.runTimesInMs = runTimesInMs;
+    }
+
+    /**
+     * @return The earliest start time out of collected run times.
+     */
+    public Date getStartTime() {
+        if (getRunTimesInMs().isEmpty()) return new Date(0);
+
+        Date startTime = null;
+        synchronized (getRunTimesInMs()) {
+            for (RunTime runTime : getRunTimesInMs()) {
+                if (null != runTime.getStartTime()) {
+                    Date currStartTime = new Date(runTime.getStartTime().getTime());
+                    if (null == startTime) {
+                        startTime = currStartTime;
+                    } else if (currStartTime.compareTo(startTime) < 0) {
+                        startTime = currStartTime;
+                    }
+                } else {
+                    startTime = new Date(0);
+                }
+            }
+        }
+        return startTime;
+    }
+
+    public RunTime getMinTimeInMs() {
+        if (getRunTimesInMs().isEmpty()) return null;
+        return Collections.min(getRunTimesInMs());
+    }
+
+    public Integer getAvgTimeInMs() {
+        if (getRunTimesInMs().isEmpty()) return null;
+
+        Integer totalTimeInMs = new Integer(0);
+        for (RunTime runTime : getRunTimesInMs()) {
+            if (null != runTime.getElapsedDurationInMs()) {
+                totalTimeInMs += runTime.getElapsedDurationInMs();
+            }
+        }
+        return totalTimeInMs / getRunTimesInMs().size();
+    }
+
+    public RunTime getMaxTimeInMs() {
+        if (getRunTimesInMs().isEmpty()) return null;
+        return Collections.max(getRunTimesInMs());
+    }
+
+    @XmlAttribute()
+    public String getThreadName() {
+        return threadName;
+    }
+
+    public void setThreadName(String threadName) {
+        this.threadName = threadName;
+    }
+    
+    private String parseThreadName(boolean getConcurrency) {
+    	if (getThreadName() == null || !getThreadName().contains(",")) return null;
+    	String[] threadNameSet = getThreadName().split(",");
+    	if (getConcurrency) {
+    		return threadNameSet[1];}
+    	else {
+    		return threadNameSet[0];
+    	}
+    }
+
+    public List<List<ResultValue>> getCsvPerformanceRepresentation(ResultUtil util) {
+        List<List<ResultValue>> rows = new ArrayList<>();
+
+        for (int i = 0; i < getRunTimesInMs().size(); i++) {
+            List<ResultValue> rowValues = new ArrayList(getRunTimesInMs().size());
+            rowValues.add(new ResultValue(util.convertNull(parseThreadName(false))));
+            rowValues.add(new ResultValue(util.convertNull(parseThreadName(true))));
+            rowValues.add(new ResultValue(String.valueOf(getRunTimesInMs().get(i).getResultRowCount())));
+            if (getRunTimesInMs().get(i).getMessage() == null) {
+                rowValues.add(new ResultValue(util.convertNull(String.valueOf(getRunTimesInMs().get(i).getElapsedDurationInMs()))));
+            } else {
+                rowValues.add(new ResultValue(util.convertNull(getRunTimesInMs().get(i).getMessage())));
+            }
+            rows.add(rowValues);
+        }
+        return rows;
+    }
+
+    public List<List<ResultValue>> getCsvFunctionalRepresentation(ResultUtil util) {
+        List<List<ResultValue>> rows = new ArrayList<>();
+
+        for (int i = 0; i < getRunTimesInMs().size(); i++) {
+            List<ResultValue> rowValues = new ArrayList<>(getRunTimesInMs().size());
+            rowValues.add(new ResultValue(util.convertNull(parseThreadName(false))));
+            rowValues.add(new ResultValue(util.convertNull(parseThreadName(true))));
+            rowValues.add(new ResultValue(util.convertNull(getRunTimesInMs().get(i).getMessage())));
+            rowValues.add(new ResultValue(util.convertNull(getRunTimesInMs().get(i).getExplainPlan())));
+            rows.add(rowValues);
+        }
+        return rows;
+    }
+
+    public int getRunCount() {
+        if (getRunTimesInMs().isEmpty()) return 0;
+        return getRunTimesInMs().size();
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/file/Extension.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/file/Extension.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/file/Extension.java
new file mode 100644
index 0000000..0df383c
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/file/Extension.java
@@ -0,0 +1,38 @@
+/*
+ * 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.result.file;
+
+public enum Extension {
+    CSV(".csv"),
+    XML(".xml"),
+    VISUALIZATION(".jpg"),
+    AGGREGATE_CSV("_aggregate" + CSV),
+    DETAILED_CSV("_detail" + CSV);
+
+    private String extension;
+
+    private Extension(String extension) {
+        this.extension = extension;
+    }
+
+    @Override
+    public String toString() {
+        return extension;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/file/Header.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/file/Header.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/file/Header.java
new file mode 100644
index 0000000..98e7b30
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/file/Header.java
@@ -0,0 +1,41 @@
+/*
+ * 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.result.file;
+
+public enum Header {
+    EMPTY(""),
+    AGGREGATE_PERFORMANCE("START_TIME,QUERY_GROUP,QUERY,TENANT_ID,AVG_MAX_TIME_MS,AVG_TIME_MS,AVG_MIN_TIME_MS,RUN_COUNT"),
+    DETAILED_BASE("BASE_TABLE_NAME,SCENARIO_NAME,ZOOKEEPER,ROW_COUNT,EXECUTION_COUNT,EXECUTION_TYPE,PHOENIX_PROPERTIES"
+            + ",START_TIME,QUERY_GROUP,QUERY,TENANT_ID,THREAD_NUMBER,CONCURRENCY_LEVEL"),
+    DETAILED_PERFORMANCE(DETAILED_BASE + ",RESULT_ROW_COUNT,RUN_TIME_MS"),
+    DETAILED_FUNCTIONAL(DETAILED_BASE + ",DIFF_STATUS,EXPLAIN_PLAN"),
+    AGGREGATE_DATA_LOAD("ZK,TABLE_NAME,ROW_COUNT,LOAD_DURATION_IN_MS"),
+    MONITOR("STAT_NAME,STAT_VALUE,TIME_STAMP");
+
+    private String header;
+
+    private Header(String header) {
+        this.header = header;
+    }
+
+    @Override
+    public String toString() {
+        return header;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/file/ResultFileDetails.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/file/ResultFileDetails.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/file/ResultFileDetails.java
new file mode 100644
index 0000000..63b6284
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/file/ResultFileDetails.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.result.file;
+
+public enum ResultFileDetails {
+    CSV(Header.EMPTY, Extension.CSV),
+    CSV_AGGREGATE_PERFORMANCE(Header.AGGREGATE_PERFORMANCE, Extension.AGGREGATE_CSV),
+    CSV_DETAILED_PERFORMANCE(Header.DETAILED_PERFORMANCE, Extension.DETAILED_CSV),
+    CSV_DETAILED_FUNCTIONAL(Header.DETAILED_FUNCTIONAL, Extension.DETAILED_CSV),
+    CSV_AGGREGATE_DATA_LOAD(Header.AGGREGATE_DATA_LOAD, Extension.CSV),
+    CSV_MONITOR(Header.MONITOR, Extension.CSV),
+    XML(Header.EMPTY, Extension.XML),
+    IMAGE(Header.EMPTY, Extension.VISUALIZATION);
+
+    private Header header;
+    private Extension extension;
+
+    private ResultFileDetails(Header header, Extension extension) {
+        this.header = header;
+        this.extension = extension;
+    }
+
+    public Extension getExtension() {
+        return extension;
+    }
+
+    public Header getHeader() {
+        return header;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/impl/CSVResultHandler.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/impl/CSVResultHandler.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/impl/CSVResultHandler.java
new file mode 100644
index 0000000..d7248fa
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/impl/CSVResultHandler.java
@@ -0,0 +1,139 @@
+/*
+ * 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.result.impl;
+
+import org.apache.phoenix.pherf.PherfConstants;
+import org.apache.phoenix.pherf.result.file.ResultFileDetails;
+import org.apache.commons.csv.CSVFormat;
+import org.apache.commons.csv.CSVParser;
+import org.apache.commons.csv.CSVPrinter;
+import org.apache.commons.csv.CSVRecord;
+import org.apache.phoenix.pherf.result.Result;
+import org.apache.phoenix.pherf.result.ResultHandler;
+import org.apache.phoenix.pherf.result.ResultUtil;
+import org.apache.phoenix.pherf.result.ResultValue;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * TODO Doc this class. Note that each instance that has a non unique file name will overwrite the last
+ */
+public class CSVResultHandler implements ResultHandler {
+
+    private final ResultUtil util;
+    private final ResultFileDetails resultFileDetails;
+    private final String resultFileName;
+    private volatile CSVPrinter csvPrinter = null;
+    private volatile boolean isClosed = true;
+
+    public CSVResultHandler(String resultFileName, ResultFileDetails resultFileDetails) {
+        this(resultFileName, resultFileDetails, true);
+    }
+
+    public CSVResultHandler(String resultFileName, ResultFileDetails resultFileDetails, boolean generateFullFileName) {
+        this.util = new ResultUtil();
+        this.resultFileName = generateFullFileName ?
+                PherfConstants.RESULT_DIR + PherfConstants.PATH_SEPARATOR
+                        + PherfConstants.RESULT_PREFIX
+                        + resultFileName + util.getSuffix()
+                        + resultFileDetails.getExtension().toString()
+            : resultFileName;
+        this.resultFileDetails = resultFileDetails;
+    }
+
+    @Override
+    public synchronized void write(Result result) throws IOException {
+        util.ensureBaseResultDirExists();
+
+        open(result);
+        csvPrinter.printRecord(result.getResultValues());
+        flush();
+    }
+
+    @Override
+    public synchronized void flush() throws IOException {
+        if (csvPrinter != null) {
+            csvPrinter.flush();
+        }
+    }
+
+    @Override
+    public synchronized void close() throws IOException {
+        if (csvPrinter != null) {
+            csvPrinter.flush();
+            csvPrinter.close();
+            isClosed = true;
+        }
+    }
+
+    @Override
+    public synchronized List<Result> read() throws IOException {
+        CSVParser parser = null;
+        util.ensureBaseResultDirExists();
+        try {
+            File file = new File(resultFileName);
+            parser = CSVParser.parse(file, Charset.defaultCharset(), CSVFormat.DEFAULT);
+            List<CSVRecord> records = parser.getRecords();
+            List<Result> results = new ArrayList<>();
+            String header = null;
+            for (CSVRecord record : records) {
+
+                // First record is the CSV Header
+                if (record.getRecordNumber() == 1) {
+                    header = record.toString();
+                    continue;
+                }
+                List<ResultValue> resultValues = new ArrayList<>();
+                for (String val : record.toString().split(PherfConstants.RESULT_FILE_DELIMETER)) {
+                    resultValues.add(new ResultValue(val));
+                }
+                Result result = new Result(resultFileDetails, header, resultValues);
+                results.add(result);
+            }
+            return results;
+        } finally {
+            parser.close();
+        }
+    }
+
+    private void open(Result result) throws IOException {
+        // Check if already so we only open one writer
+        if (csvPrinter != null) {
+            return;
+        }
+        csvPrinter = new CSVPrinter(new PrintWriter(resultFileName), CSVFormat.DEFAULT);
+        csvPrinter.printRecord(result.getHeader().split(PherfConstants.RESULT_FILE_DELIMETER));
+        isClosed = false;
+    }
+
+    @Override
+    public synchronized boolean isClosed() {
+        return isClosed;
+    }
+
+    @Override
+    public ResultFileDetails getResultFileDetails() {
+        return resultFileDetails;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/impl/ImageResultHandler.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/impl/ImageResultHandler.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/impl/ImageResultHandler.java
new file mode 100644
index 0000000..6e66cf6
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/impl/ImageResultHandler.java
@@ -0,0 +1,127 @@
+/*
+ * 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.result.impl;
+
+import org.apache.phoenix.pherf.PherfConstants;
+import org.apache.phoenix.pherf.result.file.ResultFileDetails;
+import org.apache.phoenix.pherf.result.*;
+import org.jfree.chart.ChartFactory;
+import org.jfree.chart.ChartUtilities;
+import org.jfree.chart.JFreeChart;
+import org.jfree.chart.renderer.xy.StandardXYItemRenderer;
+import org.jfree.data.time.Millisecond;
+import org.jfree.data.time.TimeSeries;
+import org.jfree.data.time.TimeSeriesCollection;
+
+import java.awt.*;
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+
+public class ImageResultHandler implements ResultHandler {
+    private final String resultFileName;
+    private final ResultFileDetails resultFileDetails;
+
+    public ImageResultHandler(String resultFileName, ResultFileDetails resultFileDetails) {
+        this(resultFileName, resultFileDetails, true);
+    }
+
+    public ImageResultHandler(String resultFileName, ResultFileDetails resultFileDetails, boolean generateFullFileName) {
+        ResultUtil util = new ResultUtil();
+        this.resultFileName = generateFullFileName ?
+                PherfConstants.RESULT_DIR + PherfConstants.PATH_SEPARATOR
+                        + PherfConstants.RESULT_PREFIX
+                        + resultFileName + util.getSuffix()
+                        + resultFileDetails.getExtension().toString()
+                : resultFileName;
+        this.resultFileDetails = resultFileDetails;
+    }
+
+    @Override
+    public synchronized void write(Result result) throws Exception {
+        TimeSeriesCollection timeSeriesCollection = new TimeSeriesCollection();
+        int rowCount = 0;
+        int maxLegendCount = 20;
+        int chartDimension = 1100;
+
+        ResultValue<DataModelResult> resultValue = result.getResultValues().get(0);
+        DataModelResult dataModelResult = resultValue.getResultValue();
+
+        for (ScenarioResult scenarioResult : dataModelResult.getScenarioResult()) {
+            for (QuerySetResult querySetResult : scenarioResult.getQuerySetResult()) {
+                for (QueryResult queryResult : querySetResult.getQueryResults()) {
+                    for (ThreadTime tt : queryResult.getThreadTimes()) {
+                        TimeSeries timeSeries = new TimeSeries(queryResult.getStatement() + " :: " + tt.getThreadName());
+                        rowCount++;
+                        synchronized (tt.getRunTimesInMs()) {
+                            for (RunTime rt : tt.getRunTimesInMs()) {
+                                if (rt.getStartTime() != null) {
+                                    timeSeries.add(new Millisecond(rt.getStartTime()), rt.getElapsedDurationInMs());
+                                }
+                            }
+                        }
+                        timeSeriesCollection.addSeries(timeSeries);
+                    }
+                }
+            }
+        }
+        boolean legend = rowCount > maxLegendCount ? false : true;
+        JFreeChart chart = ChartFactory.createTimeSeriesChart(dataModelResult.getName()
+                , "Time", "Query Time (ms)", timeSeriesCollection,
+                legend, true, false);
+        StandardXYItemRenderer renderer = new StandardXYItemRenderer(StandardXYItemRenderer.SHAPES_AND_LINES);
+        chart.getXYPlot().setRenderer(renderer);
+        chart.getXYPlot().setBackgroundPaint(Color.WHITE);
+        chart.getXYPlot().setRangeGridlinePaint(Color.BLACK);
+        for (int i = 0; i < rowCount; i++) {
+            chart.getXYPlot().getRenderer().setSeriesStroke(i, new BasicStroke(3f));
+        }
+        try {
+            ChartUtilities.saveChartAsJPEG(new File(resultFileName), chart, chartDimension, chartDimension);
+        } catch (IOException e) {
+            e.printStackTrace();
+        }
+
+    }
+
+    @Override
+    public synchronized void flush() throws Exception {
+
+    }
+
+    @Override
+    public synchronized void close() throws Exception {
+
+    }
+
+    @Override
+    public List<Result> read() throws Exception {
+        return null;
+    }
+
+    @Override
+    public boolean isClosed() {
+        return false;
+    }
+
+    @Override
+    public ResultFileDetails getResultFileDetails() {
+        return resultFileDetails;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/impl/XMLResultHandler.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/impl/XMLResultHandler.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/impl/XMLResultHandler.java
new file mode 100644
index 0000000..a1d0930
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/impl/XMLResultHandler.java
@@ -0,0 +1,103 @@
+/*
+ * 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.result.impl;
+
+import org.apache.phoenix.pherf.PherfConstants;
+import org.apache.phoenix.pherf.result.file.ResultFileDetails;
+import org.apache.phoenix.pherf.result.*;
+
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.Marshaller;
+import javax.xml.bind.Unmarshaller;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+public class XMLResultHandler implements ResultHandler {
+    private final String resultFileName;
+    private final ResultFileDetails resultFileDetails;
+
+    public XMLResultHandler(String resultFileName, ResultFileDetails resultFileDetails) {
+        this(resultFileName, resultFileDetails, true);
+    }
+
+    public XMLResultHandler(String resultFileName, ResultFileDetails resultFileDetails, boolean generateFullFileName) {
+        ResultUtil util = new ResultUtil();
+        this.resultFileName = generateFullFileName ?
+                PherfConstants.RESULT_DIR + PherfConstants.PATH_SEPARATOR
+                        + PherfConstants.RESULT_PREFIX
+                        + resultFileName + util.getSuffix()
+                        + resultFileDetails.getExtension().toString()
+                : resultFileName;
+        this.resultFileDetails = resultFileDetails;
+    }
+
+    @Override
+    public synchronized void write(Result result) throws Exception {
+        FileOutputStream os = null;
+        JAXBContext jaxbContext = JAXBContext.newInstance(DataModelResult.class);
+        Marshaller jaxbMarshaller = jaxbContext.createMarshaller();
+        jaxbMarshaller.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, Boolean.TRUE);
+        try {
+            os = new FileOutputStream(resultFileName);
+            ResultValue resultValue = result.getResultValues().get(0);
+            jaxbMarshaller.marshal(resultValue.getResultValue(), os);
+        } finally {
+            if (os != null) {
+                os.flush();
+                os.close();
+            }
+        }
+    }
+
+    @Override
+    public synchronized void flush() throws IOException {
+        return;
+    }
+
+    @Override
+    public synchronized void close() throws IOException {
+        return;
+    }
+
+    @Override
+    public synchronized List<Result> read() throws Exception {
+
+        JAXBContext jaxbContext = JAXBContext.newInstance(DataModelResult.class);
+        Unmarshaller jaxbUnmarshaller = jaxbContext.createUnmarshaller();
+        File XMLfile = new File(resultFileName);
+        List<ResultValue> resultValue = new ArrayList();
+        resultValue.add(new ResultValue<>((DataModelResult) jaxbUnmarshaller.unmarshal(XMLfile)));
+        List<Result> results = new ArrayList<>();
+        results.add(new Result(ResultFileDetails.XML, null, resultValue));
+        return results;
+    }
+
+    @Override
+    public boolean isClosed() {
+        return true;
+    }
+
+    @Override
+    public ResultFileDetails getResultFileDetails() {
+        return resultFileDetails;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/rules/DataValue.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/rules/DataValue.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/rules/DataValue.java
new file mode 100644
index 0000000..9bfc0dd
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/rules/DataValue.java
@@ -0,0 +1,89 @@
+/*
+ * 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.rules;
+
+import org.apache.phoenix.pherf.configuration.DataTypeMapping;
+
+import javax.xml.bind.annotation.*;
+
+public class DataValue {
+    private DataTypeMapping type;
+    private String value;
+    private String maxValue;
+    private String minValue;
+    private int distribution;
+
+    public DataValue() {
+        super();
+    }
+
+    public DataValue(DataTypeMapping type, String value) {
+        this.type = type;
+        this.value = value;
+        this.distribution = Integer.MIN_VALUE;
+    }
+
+    public DataValue(DataValue dataValue) {
+        this(dataValue.getType(), dataValue.getValue());
+        this.setDistribution(dataValue.getDistribution());
+        this.setMinValue(dataValue.getMinValue());
+        this.setMaxValue(dataValue.getMaxValue());
+    }
+
+    public String getValue() {
+        return value;
+    }
+
+    public DataTypeMapping getType() {
+        return type;
+    }
+
+    public int getDistribution() {
+        return distribution;
+    }
+
+    @XmlAttribute()
+    public void setDistribution(int distribution) {
+        this.distribution = distribution;
+    }
+
+    public void setType(DataTypeMapping type) {
+        this.type = type;
+    }
+
+    public void setValue(String value) {
+        this.value = value;
+    }
+
+    public String getMinValue() {
+        return minValue;
+    }
+
+    public void setMinValue(String minValue) {
+        this.minValue = minValue;
+    }
+
+    public String getMaxValue() {
+        return maxValue;
+    }
+
+    public void setMaxValue(String maxValue) {
+        this.maxValue = maxValue;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/rules/RulesApplier.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/rules/RulesApplier.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/rules/RulesApplier.java
new file mode 100644
index 0000000..4801081
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/rules/RulesApplier.java
@@ -0,0 +1,377 @@
+/*
+ * 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.rules;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.math3.random.RandomDataGenerator;
+import org.apache.phoenix.pherf.PherfConstants;
+import org.apache.commons.lang.RandomStringUtils;
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.phoenix.pherf.configuration.*;
+import org.joda.time.DateTime;
+import org.joda.time.format.DateTimeFormat;
+import org.joda.time.format.DateTimeFormatter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.text.SimpleDateFormat;
+import java.util.*;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class RulesApplier {
+    private static final Logger logger = LoggerFactory.getLogger(RulesApplier.class);
+    private static final AtomicLong COUNTER = new AtomicLong(100);
+
+    // Used to bail out of random distribution if it takes too long
+    // This should never happen when distributions add up to 100
+    private static final int OH_SHIT_LIMIT = 1000;
+
+    private final Random rndNull;
+    private final Random rndVal;
+    private final RandomDataGenerator randomDataGenerator;
+
+    private final XMLConfigParser parser;
+    private final List<Map> modelList;
+
+
+    public RulesApplier(XMLConfigParser parser) {
+        this(parser, System.currentTimeMillis());
+    }
+
+    public RulesApplier(XMLConfigParser parser, long seed) {
+        this.parser = parser;
+        this.modelList = new ArrayList<Map>();
+        this.rndNull = new Random(seed);
+        this.rndVal = new Random(seed);
+        this.randomDataGenerator = new RandomDataGenerator();
+        populateModelList();
+    }
+
+    public List<Map> getModelList() {
+        return Collections.unmodifiableList(this.modelList);
+    }
+
+
+    /**
+     * Get a data value based on rules.
+     *
+     * @param scenario      {@link org.apache.phoenix.pherf.configuration.Scenario} We are getting data for
+     * @param phxMetaColumn {@link org.apache.phoenix.pherf.configuration.Column}
+     *                      From Phoenix MetaData that are
+     *                      generating data for. It defines the
+     *                      type we are trying to match.
+     * @return
+     * @throws Exception
+     */
+    public DataValue getDataForRule(Scenario scenario, Column phxMetaColumn) throws Exception {
+        // TODO Make a Set of Rules that have already been applied so that so we don't generate for every value
+
+        List<Scenario> scenarios = parser.getScenarios();
+        DataValue value = null;
+        if (scenarios.contains(scenario)) {
+            logger.debug("We found a correct Scenario");
+            // Assume the first rule map
+            Map<DataTypeMapping, List> ruleMap = modelList.get(0);
+            List<Column> ruleList = ruleMap.get(phxMetaColumn.getType());
+
+            // Make sure Column from Phoenix Metadata matches a rule column
+            if (ruleList.contains(phxMetaColumn)) {
+                // Generate some random data based on this rule
+                logger.debug("We found a correct column rule");
+                Column columnRule = getColumnForRule(ruleList, phxMetaColumn);
+
+                value = getDataValue(columnRule);
+                synchronized (value) {
+                    // Add the prefix to the value if it exists.
+                    if (columnRule.getPrefix() != null) {
+                        value.setValue(columnRule.getPrefix() + value.getValue());
+                    }
+                }
+
+            } else {
+                logger.warn("Attempted to apply rule to data, but could not find a rule to match type:"
+                                + phxMetaColumn.getType()
+                );
+            }
+
+        }
+        return value;
+    }
+
+    /**
+     * Get data value based on the supplied rule
+     *
+     * @param column {@link org.apache.phoenix.pherf.configuration.Column} Column rule to get data for
+     * @return {@link org.apache.phoenix.pherf.rules.DataValue} Container Type --> Value mapping
+     */
+    public DataValue getDataValue(Column column) throws Exception{
+        DataValue data = null;
+        int length = column.getLength();
+        int nullChance = column.getNullChance();
+        List<DataValue> dataValues = column.getDataValues();
+
+        // Return an empty value if we we fall within the configured probability
+        if ((nullChance != Integer.MIN_VALUE) && (isValueNull(nullChance))) {
+            return new DataValue(column.getType(), "");
+        }
+
+        switch (column.getType()) {
+            case VARCHAR:
+                // Use the specified data values from configs if they exist
+                if ((column.getDataValues() != null) && (column.getDataValues().size() > 0)) {
+                    data = generateDataValue(dataValues);
+                } else {
+                    Preconditions.checkArgument(length > 0, "length needs to be > 0");
+                    if (column.getDataSequence() == DataSequence.SEQUENTIAL) {
+                        data = getSequentialDataValue(column);
+                    } else {
+                        String varchar = RandomStringUtils.randomAlphanumeric(length);
+                        data = new DataValue(column.getType(), varchar);
+                    }
+                }
+                break;
+            case CHAR:
+                if ((column.getDataValues() != null) && (column.getDataValues().size() > 0)) {
+                    data = generateDataValue(dataValues);
+                } else {
+                    Preconditions.checkArgument(length > 0, "length needs to be > 0");
+                    if (column.getDataSequence() == DataSequence.SEQUENTIAL) {
+                        data = getSequentialDataValue(column);
+                    } else {
+                        String varchar = RandomStringUtils.randomAlphanumeric(length);
+                        data = new DataValue(column.getType(), varchar);
+                    }
+                }
+                break;
+            case DECIMAL:
+                if ((column.getDataValues() != null) && (column.getDataValues().size() > 0)) {
+                    data = generateDataValue(dataValues);
+                } else {
+                    int precision = column.getPrecision();
+                    double minDbl = column.getMinValue();
+                    Preconditions.checkArgument((precision > 0) && (precision <= 18), "Precision must be between 0 and 18");
+                    Preconditions.checkArgument(minDbl >= 0, "minvalue must be set in configuration");
+                    Preconditions.checkArgument(column.getMaxValue() > 0, "maxValue must be set in configuration");
+                    StringBuilder maxValueStr = new StringBuilder();
+
+                    for (int i = 0; i < precision; i++) {
+                        maxValueStr.append(9);
+                    }
+
+                    double maxDbl = Math.min(column.getMaxValue(), Double.parseDouble(maxValueStr.toString()));
+                    final double dbl = RandomUtils.nextDouble(minDbl, maxDbl);
+                    data = new DataValue(column.getType(), String.valueOf(dbl));
+                }
+                break;
+            case INTEGER:
+                if ((column.getDataValues() != null) && (column.getDataValues().size() > 0)) {
+                    data = generateDataValue(dataValues);
+                } else {
+                    int minInt = column.getMinValue();
+                    int maxInt = column.getMaxValue();
+                    Preconditions.checkArgument((minInt > 0) && (maxInt > 0), "min and max values need to be set in configuration");
+                    int intVal = RandomUtils.nextInt(minInt, maxInt);
+                    data = new DataValue(column.getType(), String.valueOf(intVal));
+                }
+                break;
+            case DATE:
+                if ((column.getDataValues() != null) && (column.getDataValues().size() > 0)) {
+                    data = generateDataValue(dataValues);
+                } else {
+                    int minYear = column.getMinValue();
+                    int maxYear = column.getMaxValue();
+                    Preconditions.checkArgument((minYear > 0) && (maxYear > 0), "min and max values need to be set in configuration");
+
+                    String dt = generateRandomDate(minYear, maxYear);
+                    data = new DataValue(column.getType(), dt);
+                }
+                break;
+            default:
+                break;
+        }
+        Preconditions.checkArgument(data != null, "Data value could not be generated for some reason. Please check configs");
+        return data;
+    }
+
+    public String generateRandomDate(int min, int max) {
+        int year = RandomUtils.nextInt(min, max);
+        int month = RandomUtils.nextInt(0, 11);
+        int day = RandomUtils.nextInt(0, 31);
+        Calendar calendar = Calendar.getInstance();
+        calendar.set(Calendar.YEAR, year);
+        calendar.set(Calendar.MONTH, month);
+        calendar.set(Calendar.DAY_OF_MONTH, day);
+        SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS z");
+
+        return df.format(calendar.getTime());
+    }
+
+    public String generateRandomDate(String min, String max) throws Exception {
+        DateTimeFormatter fmtr = DateTimeFormat.forPattern(PherfConstants.DEFAULT_DATE_PATTERN);
+        DateTime minDt = fmtr.parseDateTime(min);
+        DateTime maxDt = fmtr.parseDateTime(max);
+        DateTime dt;
+        // Get Ms Date between min and max
+        synchronized (randomDataGenerator) {
+            long rndLong = randomDataGenerator.nextLong(minDt.getMillis(), maxDt.getMillis());
+            dt = new DateTime(rndLong, minDt.getZone());
+        }
+
+        return fmtr.print(dt);
+    }
+
+    /**
+     * Given an int chance [0-100] inclusive, this method will return true if a winner is selected, otherwise false.
+     *
+     * @param chance Percentage as an int while number.
+     * @return boolean if we pick a number within range
+     */
+    private boolean isValueNull(int chance) {
+        return (rndNull.nextInt(100) < chance);
+    }
+
+    private DataValue generateDataValue(List<DataValue> values) throws Exception{
+        DataValue generatedDataValue = null;
+        int sum = 0, count = 0;
+
+        // Verify distributions add up to 100 if they exist
+        for (DataValue value : values) {
+            int dist = value.getDistribution();
+            sum += dist;
+        }
+        Preconditions.checkArgument((sum == 100) || (sum == 0), "Distributions need to add up to 100 or not exist.");
+
+        // Spin the wheel until we get a value.
+        while (generatedDataValue == null) {
+
+            // Give an equal chance at picking any one rule to test
+            // This prevents rules at the beginning of the list from getting more chances to get picked
+            int rndIndex = rndVal.nextInt(values.size());
+            DataValue valueRule = values.get(rndIndex);
+
+            generatedDataValue = generateDataValue(valueRule);
+
+            // While it's possible to get here if you have a bunch of really small distributions,
+            // It's just really unlikely. This is just a safety just so we actually pick a value.
+            if(count++ == OH_SHIT_LIMIT){
+                logger.info("We generated a value from hitting our OH_SHIT_LIMIT: " + OH_SHIT_LIMIT);
+                generatedDataValue = valueRule;
+            }
+
+        }
+        return generatedDataValue;
+    }
+
+    private DataValue generateDataValue(final DataValue valueRule) throws Exception{
+        DataValue retValue = new DataValue(valueRule);
+
+        // Path taken when configuration specifies a specific value to be taken with the <value> tag
+        if (valueRule.getValue() != null) {
+            int chance = (valueRule.getDistribution() == 0) ? 100 : valueRule.getDistribution();
+            return (rndVal.nextInt(100) <= chance) ? retValue : null;
+        }
+
+        // Later we can add support fo other data types if needed.Right now, we just do this for dates
+        Preconditions.checkArgument((retValue.getMinValue() != null) || (retValue.getMaxValue() != null), "Both min/maxValue tags must be set if value tag is not used");
+        Preconditions.checkArgument((retValue.getType() == DataTypeMapping.DATE), "Currently on DATE is supported for ranged random values");
+
+        retValue.setValue(generateRandomDate(retValue.getMinValue(), retValue.getMaxValue()));
+
+        return retValue;
+    }
+
+    /**
+     * Top level {@link java.util.List} {@link java.util.Map}. This will likely only have one entry until we have
+     * multiple files.
+     * <p/>
+     * <p/>
+     * Each Map entry in the List is:
+     * {@link java.util.Map} of
+     * {@link org.apache.phoenix.pherf.configuration.DataTypeMapping} -->
+     * List of {@link org.apache.phoenix.pherf.configuration.Column
+     * Build the initial Map with all the general rules.
+     * These are contained in:
+     * <datamode><datamapping><column>...</column></datamapping></datamode>
+     * <p/>
+     * <p/>
+     * Unsupported until V2
+     * Build the overrides by appending them to the list of rules that match the column type
+     */
+    private void populateModelList() {
+        if (!modelList.isEmpty()) {
+            return;
+        }
+
+        // Support for multiple models, but rules are only relevant each model
+        for (DataModel model : parser.getDataModels()) {
+
+            // Step 1
+            final Map<DataTypeMapping, List> ruleMap = new HashMap<DataTypeMapping, List>();
+            for (Column column : model.getDataMappingColumns()) {
+                List<Column> cols;
+                DataTypeMapping type = column.getType();
+                if (ruleMap.containsKey(type)) {
+                    ruleMap.get(type).add(column);
+                } else {
+                    cols = new LinkedList<Column>();
+                    cols.add(column);
+                    ruleMap.put(type, cols);
+                }
+            }
+
+            this.modelList.add(ruleMap);
+        }
+    }
+
+    private Column getColumnForRule(List<Column> ruleList, Column phxMetaColumn) {
+
+        // Column pointer to head of list
+        Column ruleAppliedColumn = new Column(ruleList.get(0));
+
+        // Then we apply each rule override as a mutation to the column
+        for (Column columnRule : ruleList) {
+
+            // Check if user defined column rules match the column data type we are generating
+            // We don't want to apply the rule if name doesn't match the column from Phoenix
+            if (columnRule.isUserDefined()
+                    && !columnRule.getName().equals(phxMetaColumn.getName())) {
+                continue;
+            }
+            ruleAppliedColumn.mutate(columnRule);
+        }
+
+        return ruleAppliedColumn;
+    }
+
+    /**
+     * Add a numerically increasing counter onto the and of a random string.
+     * Incremented counter should be thread safe.
+     *
+     * @param column {@link org.apache.phoenix.pherf.configuration.Column}
+     * @return {@link org.apache.phoenix.pherf.rules.DataValue}
+     */
+    private DataValue getSequentialDataValue(Column column) {
+        DataValue data = null;
+        long inc = COUNTER.getAndIncrement();
+        String strInc = String.valueOf(inc);
+        String varchar = RandomStringUtils.randomAlphanumeric(column.getLength() - strInc.length());
+        data = new DataValue(column.getType(), strInc + varchar);
+        return data;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/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
new file mode 100644
index 0000000..234dd14
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/schema/SchemaReader.java
@@ -0,0 +1,97 @@
+/*
+ * 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.schema;
+
+import org.apache.phoenix.pherf.PherfConstants;
+import org.apache.phoenix.pherf.exception.FileLoaderException;
+import org.apache.phoenix.pherf.util.PhoenixUtil;
+import org.apache.phoenix.pherf.util.ResourceList;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.InputStreamReader;
+import java.nio.file.Path;
+import java.sql.Connection;
+import java.util.Collection;
+
+public class SchemaReader {
+    private static final Logger logger = LoggerFactory.getLogger(SchemaReader.class);
+    private final PhoenixUtil pUtil = new PhoenixUtil();
+    private Collection<Path> resourceList;
+    private final String searchPattern;
+    private final ResourceList resourceUtil;
+
+    /**
+     * Used for testing search Pattern
+     * @param searchPattern {@link java.util.regex.Pattern} that matches a resource on the CP
+     * @throws Exception
+     */
+    public SchemaReader(final String searchPattern) throws Exception {
+        this.searchPattern = searchPattern;
+        this.resourceUtil = new ResourceList(PherfConstants.RESOURCE_DATAMODEL);
+        read();
+    }
+
+    public Collection<Path> getResourceList() {
+        return resourceList;
+    }
+
+    public void applySchema() throws Exception {
+        Connection connection = null;
+        try {
+            connection = pUtil.getConnection();
+            for (Path file : resourceList) {
+                logger.info("\nApplying schema to file: " + file);
+                pUtil.executeStatement(resourceToString(file), connection);
+            }
+        } finally {
+            if (connection != null) {
+                connection.close();
+            }
+        }
+    }
+
+    public String resourceToString(final Path file) throws Exception {
+        String fName = PherfConstants.RESOURCE_DATAMODEL + "/" + file.getFileName().toString();
+        BufferedReader br = new BufferedReader(new InputStreamReader(this.getClass().getResourceAsStream(fName)));
+        StringBuffer sb = new StringBuffer();
+
+        String line;
+        while ((line = br.readLine()) != null) {
+            sb.append(line);
+        }
+
+        return sb.toString();
+    }
+
+    private void read() throws Exception {
+        logger.debug("Trying to match resource pattern: " + searchPattern);
+        System.out.println("Trying to match resource pattern: " + searchPattern);
+
+        resourceList = null;
+        resourceList = resourceUtil.getResourceList(searchPattern);
+        logger.info("File resourceList Loaded: " + resourceList);
+        System.out.println("File resourceList Loaded: " + resourceList);
+        if (resourceList.isEmpty()) {
+            throw new FileLoaderException("Could not load Schema Files");
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/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
new file mode 100644
index 0000000..7836ea3
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/util/PhoenixUtil.java
@@ -0,0 +1,199 @@
+/*
+ * 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.util;
+
+import org.apache.phoenix.pherf.PherfConstants;
+import org.apache.phoenix.pherf.configuration.Column;
+import org.apache.phoenix.pherf.configuration.DataTypeMapping;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.sql.*;
+import java.util.*;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+// TODO This class needs to be cleanup up a bit. I just wanted to get an initial placeholder in.
+public class PhoenixUtil {
+	private static final Logger logger = LoggerFactory.getLogger(PhoenixUtil.class);
+	private static String zookeeper;
+	private static int rowCountOverride = 0;
+	
+    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 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 boolean executeStatement(String sql) throws Exception {
+        Connection connection = null;
+        boolean result = false;
+        try {
+            connection = getConnection();
+            result = executeStatement(sql, connection);
+        } finally {
+            if (connection != null) {
+                connection.close();
+            }
+        }
+        return result;
+    }
+    
+    public boolean executeStatement(String sql, Connection connection) {
+    	boolean result = false;
+        PreparedStatement preparedStatement = null;
+        try {
+            preparedStatement = connection.prepareStatement(sql);
+            result = preparedStatement.execute();
+            connection.commit();
+        } catch (SQLException e) {
+            e.printStackTrace();
+        } finally {
+            try {
+                preparedStatement.close();
+            } catch (SQLException e) {
+                e.printStackTrace();
+            }
+        }
+        return result;
+    }
+    
+    public boolean executeStatement(PreparedStatement preparedStatement, Connection connection) {
+    	boolean result = false;
+        try {
+            result = preparedStatement.execute();
+            connection.commit();
+        } catch (SQLException e) {
+            e.printStackTrace();
+        }
+        return result;
+    }
+
+    public ResultSet executeQuery(PreparedStatement preparedStatement, Connection connection) {
+        ResultSet resultSet = null;
+        try {
+            resultSet = preparedStatement.executeQuery();
+        } catch (SQLException e) {
+            e.printStackTrace();
+        }
+        return resultSet;
+    }
+    
+    /**
+     * Delete existing tables with schema name set as {@link PherfConstants#PHERF_SCHEMA_NAME} with regex comparison 
+     * 
+     * @param regexMatch
+     * @throws SQLException
+     * @throws Exception
+     */
+    public void deleteTables(String regexMatch) throws SQLException, Exception {
+    	regexMatch = regexMatch.toUpperCase().replace("ALL", ".*");
+    	Connection conn = getConnection();
+    	try {
+        	ResultSet resultSet = getTableMetaData(PherfConstants.PHERF_SCHEMA_NAME, null, conn);
+	    	while (resultSet.next()) {
+	    		String tableName = resultSet.getString("TABLE_SCHEM") == null ? resultSet.getString("TABLE_NAME") : 
+	    						   resultSet.getString("TABLE_SCHEM") + "." + resultSet.getString("TABLE_NAME");
+	    		if (tableName.matches(regexMatch)) {
+		    		logger.info("\nDropping " + tableName);
+		    		executeStatement("DROP TABLE " + tableName + " CASCADE", conn);
+	    		}
+	    	}
+    	} finally {
+    		conn.close();
+    	}
+    }
+    
+    public ResultSet getTableMetaData(String schemaName, String tableName, Connection connection) throws SQLException {
+    	DatabaseMetaData dbmd = connection.getMetaData();
+    	ResultSet resultSet = dbmd.getTables(null, schemaName, tableName, null);
+    	return resultSet;
+    }
+    
+    public ResultSet getColumnsMetaData(String schemaName, String tableName, Connection connection) throws SQLException {
+    	DatabaseMetaData dbmd = connection.getMetaData();
+    	ResultSet resultSet = dbmd.getColumns(null, schemaName, tableName, null);
+    	return resultSet;
+    }
+    
+    public synchronized List<Column> getColumnsFromPhoenix(String schemaName, String tableName, Connection connection) throws SQLException {
+    	List<Column> columnList = new ArrayList<Column>();
+    	ResultSet resultSet = null;
+    	try {
+    		resultSet = getColumnsMetaData(schemaName, tableName, connection);
+    		while (resultSet.next()) {
+    			Column column = new Column();
+    	        column.setName(resultSet.getString("COLUMN_NAME"));
+    	        column.setType(DataTypeMapping.valueOf(resultSet.getString("TYPE_NAME")));
+    	        column.setLength(resultSet.getInt("COLUMN_SIZE"));
+    	        columnList.add(column);
+   	        }
+    	} finally {
+    		if (null != resultSet) { 
+    			resultSet.close();
+    		}
+    	}
+    	
+    	return Collections.unmodifiableList(columnList);
+    }
+    
+	public static String getZookeeper() {
+		return zookeeper;
+	}
+
+	public static void setZookeeper(String zookeeper) {
+		logger.info("Setting zookeeper: " + zookeeper);
+		PhoenixUtil.zookeeper = zookeeper;
+	}
+	
+	public static int getRowCountOverride() {
+		return rowCountOverride;
+	}
+	
+	public static void setRowCountOverride(int rowCountOverride) {
+		PhoenixUtil.rowCountOverride = rowCountOverride;
+	}
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/36b88651/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
new file mode 100644
index 0000000..2d6e77d
--- /dev/null
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/util/ResourceList.java
@@ -0,0 +1,214 @@
+/*
+ * 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.util;
+
+import org.apache.phoenix.pherf.PherfConstants;
+import org.apache.phoenix.pherf.exception.PherfException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URI;
+import java.net.URL;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.*;
+import java.util.regex.Pattern;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipException;
+import java.util.zip.ZipFile;
+
+/**
+ * list resources available from the classpath @ *
+ */
+public class ResourceList {
+    private static final Logger logger = LoggerFactory.getLogger(ResourceList.class);
+    private final String rootResourceDir;
+
+    public ResourceList() {
+        this("/");
+    }
+
+    public ResourceList(String rootResourceDir) {
+        this.rootResourceDir = rootResourceDir;
+    }
+
+    public Collection<Path> getResourceList(final String pattern) throws Exception {
+        Properties properties = getProperties();
+
+        // Include files from config directory
+        Collection<Path> paths = getResourcesPaths(Pattern.compile(pattern));
+
+
+        return paths;
+    }
+
+    /**
+     * for all elements of java.class.path get a Collection of resources Pattern
+     * pattern = Pattern.compile(".*"); gets all resources
+     *
+     * @param pattern the pattern to match
+     * @return the resources in the order they are found
+     */
+    private Collection<Path> getResourcesPaths(
+            final Pattern pattern) throws Exception {
+
+        final String classPath = System.getProperty("java.class.path", ".");
+        final String[] classPathElements = classPath.split(":");
+        List<String> strResources = new ArrayList<>();
+        Collection<Path> paths = new ArrayList<>();
+
+        // TODO Make getResourcesPaths() return the URLs directly instead of converting them
+        // Get resources as strings.
+        for (final String element : classPathElements) {
+            strResources.addAll(getResources(element, pattern));
+        }
+
+        // Convert resources to URL
+        for (String resource : strResources) {
+            URL url = null;
+            URI uri = null;
+            Path path = null;
+
+            String rName = rootResourceDir + resource;
+
+            logger.debug("Trying with the root append.");
+            url = ResourceList.class.getResource(rName);
+            if (url == null) {
+                logger.debug("Failed! Must be using a jar. Trying without the root append.");
+                url = ResourceList.class.getResource(resource);
+
+                if (url == null) {
+                    throw new PherfException("Could not load resources: " + rName);
+                }
+                final String[] splits = url.toString().split("!");
+                uri = URI.create(splits[0]);
+                path = (splits.length < 2) ? Paths.get(uri) : Paths.get(splits[1]);
+            } else {
+                path = Paths.get(url.toURI());
+            }
+            logger.debug("Found the correct resource: " + path.toString());
+            paths.add(path);
+        }
+
+        return paths;
+    }
+
+    public Properties getProperties() throws Exception {
+        return getProperties(PherfConstants.PHERF_PROPERTIES);
+    }
+
+    public Properties getProperties(final String fileName) throws Exception {
+        Properties pherfProps = new Properties();
+        InputStream is = null;
+        try {
+            is = getClass().getClassLoader().getResourceAsStream(fileName);
+            pherfProps.load(is);
+        } finally {
+            if (is != null) {
+                is.close();
+            }
+        }
+        return pherfProps;
+    }
+
+    /**
+     * Utility method to check if base result dir exists
+     */
+    public void ensureBaseDirExists(String directory) {
+        File baseDir = new File(directory);
+        if (!baseDir.exists()) {
+            boolean made = baseDir.mkdir();
+            if (!made) {
+                logger.error("Could not make directory:" + directory);
+            }
+        }
+    }
+
+    private Collection<String> getResources(
+            final String element,
+            final Pattern pattern) {
+        final List<String> retVal = new ArrayList<>();
+        if (element.equals("")) {
+            return retVal;
+        }
+        final File file = new File(element);
+        if (file.isDirectory()) {
+            retVal.addAll(getResourcesFromDirectory(file, pattern));
+        } else {
+            retVal.addAll(getResourcesFromJarFile(file, pattern));
+        }
+        return retVal;
+    }
+
+    private Collection<String> getResourcesFromJarFile(
+            final File file,
+            final Pattern pattern) {
+        final List<String> retVal = new ArrayList<>();
+        ZipFile zf;
+        try {
+            zf = new ZipFile(file);
+        } catch (final ZipException e) {
+            throw new Error(e);
+        } catch (final IOException e) {
+            throw new Error(e);
+        }
+        final Enumeration e = zf.entries();
+        while (e.hasMoreElements()) {
+            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);
+            if (accept) {
+                logger.debug("Adding File from Jar: " + fileName);
+                retVal.add("/" + fileName);
+            }
+        }
+        try {
+            zf.close();
+        } catch (final IOException e1) {
+            throw new Error(e1);
+        }
+        return retVal;
+    }
+
+    private Collection<String> getResourcesFromDirectory(
+            final File directory,
+            final Pattern pattern) {
+        final ArrayList<String> retval = new ArrayList<String>();
+        final File[] fileList = directory.listFiles();
+        for (final File file : fileList) {
+            if (file.isDirectory()) {
+                retval.addAll(getResourcesFromDirectory(file, pattern));
+            } else {
+                final String fileName = file.getName();
+                final boolean accept = pattern.matcher(file.toString()).matches();
+                if (accept) {
+                    logger.debug("Adding File from directory: " + fileName);
+                    retval.add("/" + fileName);
+                }
+            }
+        }
+        return retval;
+    }
+}
\ No newline at end of file