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:32 UTC

[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.

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