You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ja...@apache.org on 2014/04/10 09:07:45 UTC

[2/3] git commit: PHOENIX-896 Improve PhoenixRuntime cmdline parsing

PHOENIX-896 Improve PhoenixRuntime cmdline parsing

Use commons-cli parameter parsing to parse command-line parameters
instead of using custom parsing code. Also add some testing of
various scenarios in the command-line parsing.

This commit introduces a small backwards-incompatibility, in that
the CSV field separator, quote character, and escape character are
now supplied as separate parameters instead of a single parameter.


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

Branch: refs/heads/local-index
Commit: ec3e840054e7f6f9bf5693ee8b99505cb2ee8762
Parents: 13bd857
Author: Gabriel Reid <ga...@ngdata.com>
Authored: Wed Mar 26 21:13:19 2014 +0100
Committer: Gabriel Reid <ga...@ngdata.com>
Committed: Wed Apr 9 09:16:24 2014 +0200

----------------------------------------------------------------------
 .../phoenix/end2end/CSVCommonsLoaderIT.java     |   8 +-
 .../apache/phoenix/util/CSVCommonsLoader.java   |  88 +++---
 .../org/apache/phoenix/util/PhoenixRuntime.java | 312 +++++++++++++------
 .../apache/phoenix/util/PhoenixRuntimeTest.java |  57 ++++
 4 files changed, 313 insertions(+), 152 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/ec3e8400/phoenix-core/src/it/java/org/apache/phoenix/end2end/CSVCommonsLoaderIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CSVCommonsLoaderIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CSVCommonsLoaderIT.java
index 543142f..28142bc 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CSVCommonsLoaderIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CSVCommonsLoaderIT.java
@@ -153,7 +153,7 @@ public class CSVCommonsLoaderIT extends BaseHBaseManagedTimeIT {
 
             // Upsert TDV file
             CSVCommonsLoader csvUtil = new CSVCommonsLoader(conn, STOCK_TABLE,Collections.<String> emptyList()
-                    , true,Arrays.asList("\t", "0", "0"), CSVCommonsLoader.DEFAULT_ARRAY_ELEMENT_SEPARATOR);
+                    , true, '\t', '"', null, CSVCommonsLoader.DEFAULT_ARRAY_ELEMENT_SEPARATOR);
             csvUtil.upsert(new StringReader(STOCK_TDV_VALUES_WITH_HEADER));
 
             // Compare Phoenix ResultSet with CSV file content
@@ -195,8 +195,8 @@ public class CSVCommonsLoaderIT extends BaseHBaseManagedTimeIT {
 
             // Upsert CSV file
             CSVCommonsLoader csvUtil = new CSVCommonsLoader(conn, STOCK_TABLE,
-                    Arrays.<String> asList(STOCK_COLUMNS), true, Arrays.asList(
-                    "1", "2", "3"), CSVCommonsLoader.DEFAULT_ARRAY_ELEMENT_SEPARATOR);
+                    Arrays.<String> asList(STOCK_COLUMNS), true,
+                    '1', '2', '3', CSVCommonsLoader.DEFAULT_ARRAY_ELEMENT_SEPARATOR);
             csvUtil.upsert(new StringReader(STOCK_CSV_VALUES_WITH_DELIMITER));
 
             // Compare Phoenix ResultSet with CSV file content
@@ -575,7 +575,7 @@ public class CSVCommonsLoaderIT extends BaseHBaseManagedTimeIT {
 
             // Upsert CSV file
             CSVCommonsLoader csvUtil = new CSVCommonsLoader(conn, "ARRAY_TABLE",
-                    Collections.<String> emptyList(), true, null, "!");
+                    null, true, ',', '"', null, "!");
             csvUtil.upsert(
                     new StringReader("ID,VALARRAY\n"
                             + "1,2!3!4\n"));

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/ec3e8400/phoenix-core/src/main/java/org/apache/phoenix/util/CSVCommonsLoader.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/CSVCommonsLoader.java b/phoenix-core/src/main/java/org/apache/phoenix/util/CSVCommonsLoader.java
index 0292aae..8d67e20 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/CSVCommonsLoader.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/CSVCommonsLoader.java
@@ -25,10 +25,10 @@ import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import com.google.common.collect.ImmutableMap;
 import org.apache.commons.csv.CSVFormat;
 import org.apache.commons.csv.CSVParser;
 import org.apache.commons.csv.CSVRecord;
@@ -53,24 +53,29 @@ public class CSVCommonsLoader {
 
     public static final String DEFAULT_ARRAY_ELEMENT_SEPARATOR = ":";
 
+    private static final Map<Character,Character> CTRL_CHARACTER_TABLE =
+            ImmutableMap.<Character,Character>builder()
+                        .put('1', '\u0001')
+                        .put('2', '\u0002')
+                        .put('3', '\u0003')
+                        .put('4', '\u0004')
+                        .put('5', '\u0005')
+                        .put('6', '\u0006')
+                        .put('7', '\u0007')
+                        .put('8', '\u0008')
+                        .put('9', '\u0009')
+                        .build();
+
     private final PhoenixConnection conn;
     private final String tableName;
     private final List<String> columns;
     private final boolean isStrict;
-    boolean userSuppliedMetaCharacters = false;
-    private final List<String> customMetaCharacters;
+    private final char fieldDelimiter;
+    private final char quoteCharacter;
+    private final Character escapeCharacter;
     private PhoenixHeaderSource headerSource = PhoenixHeaderSource.FROM_TABLE;
     private final CSVFormat format;
-    private final Map<String,Character> ctrlTable = new HashMap<String,Character>() {
-        {   put("1",'\u0001');
-            put("2",'\u0002');
-            put("3",'\u0003');
-            put("4",'\u0004');
-            put("5",'\u0005');
-            put("6",'\u0006');
-            put("7",'\u0007');
-            put("8",'\u0008');
-            put("9",'\u0009');}};
+
 
     private final String arrayElementSeparator;
 
@@ -82,26 +87,19 @@ public class CSVCommonsLoader {
 
     public CSVCommonsLoader(PhoenixConnection conn, String tableName,
             List<String> columns, boolean isStrict) {
-        this(conn, tableName, columns, isStrict, null, DEFAULT_ARRAY_ELEMENT_SEPARATOR);
+        this(conn, tableName, columns, isStrict, ',', '"', null, DEFAULT_ARRAY_ELEMENT_SEPARATOR);
     }
 
     public CSVCommonsLoader(PhoenixConnection conn, String tableName,
-            List<String> columns, boolean isStrict, List<String> customMetaCharacters, String arrayElementSeparator) {
+            List<String> columns, boolean isStrict, char fieldDelimiter, char quoteCharacter,
+            Character escapeCharacter, String arrayElementSeparator) {
         this.conn = conn;
         this.tableName = tableName;
         this.columns = columns;
         this.isStrict = isStrict;
-        this.customMetaCharacters = customMetaCharacters;
-        if (customMetaCharacters==null || customMetaCharacters.size()==0) {
-            userSuppliedMetaCharacters=false;
-        } else if (customMetaCharacters.size()==3) {
-            userSuppliedMetaCharacters=true;
-        }
-        else{
-            throw new IllegalArgumentException(
-                    String.format("customMetaCharacters must have no elements or three elements. Supplied value is %s",
-                            buildStringFromList(customMetaCharacters)));
-        }
+        this.fieldDelimiter = fieldDelimiter;
+        this.quoteCharacter = quoteCharacter;
+        this.escapeCharacter = escapeCharacter;
 
         // implicit in the columns value.
         if (columns !=null && !columns.isEmpty()) {
@@ -131,25 +129,14 @@ public class CSVCommonsLoader {
      */
     private CSVFormat buildFormat() {
         CSVFormat format = CSVFormat.DEFAULT
-                .withIgnoreEmptyLines(true);
-        if (userSuppliedMetaCharacters) {
-            // list error checking handled in constructor above.
-            // use 0 to keep default setting
-            String delimiter = customMetaCharacters.get(0);
-            String quote = customMetaCharacters.get(1);
-            String escape = customMetaCharacters.get(2);
-
-            if (!"0".equals(delimiter)) {
-                format = format.withDelimiter(getCustomMetaCharacter(delimiter));
-            }
-            if (!"0".equals(quote)) {
-                format = format.withQuoteChar(getCustomMetaCharacter(quote));
-            }
-            if (!"0".equals(quote)) {
-                format = format.withEscape(getCustomMetaCharacter(escape));
-            }
+                .withIgnoreEmptyLines(true)
+                .withDelimiter(asControlCharacter(fieldDelimiter))
+                .withQuoteChar(asControlCharacter(quoteCharacter));
 
+        if (escapeCharacter != null) {
+            format = format.withEscape(asControlCharacter(escapeCharacter));
         }
+
         switch(headerSource) {
         case FROM_TABLE:
             // obtain headers from table, so format should not expect a header.
@@ -170,11 +157,18 @@ public class CSVCommonsLoader {
     }
 
 
-    public char getCustomMetaCharacter(String field) {
-        if(this.ctrlTable.containsKey(field)) {
-            return this.ctrlTable.get(field);
+    /**
+     * Translate a field separator, escape character, or phrase delimiter into a control character
+     * if it is a single digit other than 0.
+     *
+     * @param delimiter
+     * @return
+     */
+    public static char asControlCharacter(char delimiter) {
+        if(CTRL_CHARACTER_TABLE.containsKey(delimiter)) {
+            return CTRL_CHARACTER_TABLE.get(delimiter);
         } else {
-            return field.charAt(0);
+            return delimiter;
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/ec3e8400/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java b/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
index 27c0c2a..a49627c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
@@ -31,6 +31,15 @@ import java.util.List;
 import java.util.Properties;
 import java.util.StringTokenizer;
 
+import com.google.common.base.Splitter;
+import com.google.common.collect.ImmutableList;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.commons.cli.PosixParser;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.Mutation;
@@ -101,36 +110,10 @@ public class PhoenixRuntime {
      */
     public final static String CONNECTIONLESS = "none";
     
-    private static final String TABLE_OPTION = "-t";
-    private static final String HEADER_OPTION = "-h";
-    private static final String STRICT_OPTION = "-s";
-    private static final String CSV_OPTION = "-d";
-    private static final String ARRAY_ELEMENT_SEP_OPTION = "-a";
     private static final String HEADER_IN_LINE = "in-line";
     private static final String SQL_FILE_EXT = ".sql";
     private static final String CSV_FILE_EXT = ".csv";
 
-    private static void usageError() {
-        System.err.println("Usage: psql [-t table-name] [-h comma-separated-column-names | in-line] [-d field-delimiter-char quote-char escape-char]<zookeeper>  <path-to-sql-or-csv-file>...\n" +
-                "  By default, the name of the CSV file (case insensitive) is used to determine the Phoenix table into which the CSV data is loaded\n" +
-                "  and the ordinal value of the columns determines the mapping.\n" +
-                "  -t overrides the table into which the CSV data is loaded and is case sensitive.\n" +
-                "  -h overrides the column names to which the CSV data maps and is case sensitive.\n" +
-                "     A special value of in-line indicating that the first line of the CSV file\n" +
-                "     determines the column to which the data maps.\n" +
-                "  -s uses strict mode by throwing an exception if a column name doesn't match during CSV loading.\n" +
-                "  -d uses custom delimiters for CSV loader, need to specify single char for field delimiter, phrase delimiter, and escape char.\n" +
-                "     number is NOT usually a delimiter and shall be taken as 1 -> ctrl A, 2 -> ctrl B ... 9 -> ctrl I. \n" +
-                "  -a define the array element separator, defaults to ':'\n" +
-                "Examples:\n" +
-                "  psql localhost my_ddl.sql\n" +
-                "  psql localhost my_ddl.sql my_table.csv\n" +
-                "  psql -t MY_TABLE my_cluster:1825 my_table2012-Q3.csv\n" +
-                "  psql -t MY_TABLE -h COL1,COL2,COL3 my_cluster:1825 my_table2012-Q3.csv\n" +
-                "  psql -t MY_TABLE -h COL1,COL2,COL3 -d 1 2 3 my_cluster:1825 my_table2012-Q3.csv\n"
-        );
-        System.exit(-1);
-    }
     /**
      * Provides a mechanism to run SQL scripts against, where the arguments are:
      * 1) connection URL string
@@ -140,90 +123,40 @@ public class PhoenixRuntime {
      * increment timestamp value.
      */
     public static void main(String [] args) {
-        if (args.length < 2) {
-            usageError();
-        }
+
+        ExecutionCommand execCmd = ExecutionCommand.parseArgs(args);
+        String jdbcUrl = JDBC_PROTOCOL + JDBC_PROTOCOL_SEPARATOR + execCmd.getConnectionString();
+
         PhoenixConnection conn = null;
         try {
-            String tableName = null;
-            List<String> columns = null;
-            boolean isStrict = false;
-            String arrayElementSeparator = CSVCommonsLoader.DEFAULT_ARRAY_ELEMENT_SEPARATOR;
-            List<String> customMetaCharacters = new ArrayList<String>();
-
-            int i = 0;
-            for (; i < args.length; i++) {
-                if (TABLE_OPTION.equals(args[i])) {
-                    if (++i == args.length || tableName != null) {
-                        usageError();
-                    }
-                    tableName = args[i];
-                } else if (HEADER_OPTION.equals(args[i])) {
-                    if (++i >= args.length || columns != null) {
-                        usageError();
-                    }
-                    String header = args[i];
-                    if (HEADER_IN_LINE.equals(header)) {
-                        columns = Collections.emptyList();
-                    } else {
-                        columns = Lists.newArrayList();
-                        StringTokenizer tokenizer = new StringTokenizer(header,",");
-                        while(tokenizer.hasMoreTokens()) {
-                            columns.add(tokenizer.nextToken());
-                        }
-                    }
-                } else if (STRICT_OPTION.equals(args[i])) {
-                    isStrict = true;
-                } else if (CSV_OPTION.equals(args[i])) {
-                    for(int j=0; j < 3; j++) {
-                        if(args[++i].length()==1){
-                            customMetaCharacters.add(args[i]);
-                        } else {
-                            usageError();
-                        }
-                    }
-                } else if (ARRAY_ELEMENT_SEP_OPTION.equals(args[i])) {
-                    arrayElementSeparator = args[++i];
-                } else {
-                    break;
-                }
-            }
-            if (i == args.length) {
-                usageError();
-            }
-            
             Properties props = new Properties();
-            String connectionUrl = JDBC_PROTOCOL + JDBC_PROTOCOL_SEPARATOR + args[i++];
-            conn = DriverManager.getConnection(connectionUrl, props).unwrap(PhoenixConnection.class);
-            
-            for (; i < args.length; i++) {
-                String fileName = args[i];
-                if (fileName.endsWith(SQL_FILE_EXT)) {
-               		PhoenixRuntime.executeStatements(conn, new FileReader(args[i]), Collections.emptyList());
-                } else if (fileName.endsWith(CSV_FILE_EXT)) {
+            conn = DriverManager.getConnection(jdbcUrl, props)
+                    .unwrap(PhoenixConnection.class);
+
+            for (String inputFile : execCmd.getInputFiles()) {
+                if (inputFile.endsWith(SQL_FILE_EXT)) {
+                    PhoenixRuntime.executeStatements(conn,
+                            new FileReader(inputFile), Collections.emptyList());
+                } else if (inputFile.endsWith(CSV_FILE_EXT)) {
+
+                    String tableName = execCmd.getTableName();
                     if (tableName == null) {
-                        tableName = SchemaUtil.normalizeIdentifier(fileName.substring(fileName.lastIndexOf(File.separatorChar) + 1, fileName.length()-CSV_FILE_EXT.length()));
+                        tableName = SchemaUtil.normalizeIdentifier(
+                                inputFile.substring(inputFile.lastIndexOf(File.separatorChar) + 1,
+                                        inputFile.length() - CSV_FILE_EXT.length()));
                     }
-                    CSVCommonsLoader csvLoader = 
-                    		new CSVCommonsLoader(conn, tableName, columns, isStrict, customMetaCharacters, arrayElementSeparator);
-                    csvLoader.upsert(fileName);
-                } else {
-                    usageError();
-                }
-                Long scn = conn.getSCN();
-                // If specifying SCN, increment it between processing files to allow
-                // for later files to see earlier files tables.
-                if (scn != null) {
-                    scn++;
-                    props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, scn.toString());
-                    conn.close();
-                    conn = DriverManager.getConnection(connectionUrl, props).unwrap(PhoenixConnection.class);
+                    CSVCommonsLoader csvLoader =
+                            new CSVCommonsLoader(conn, tableName, execCmd.getColumns(),
+                                    execCmd.isStrict(), execCmd.getFieldDelimiter(),
+                                    execCmd.getQuoteCharacter(), execCmd.getEscapeCharacter(),
+                                    execCmd.getArrayElementSeparator());
+                    csvLoader.upsert(inputFile);
                 }
             }
         } catch (Throwable t) {
             t.printStackTrace();
         } finally {
-            if(conn != null) {
+            if (conn != null) {
                 try {
                     conn.close();
                 } catch (SQLException e) {
@@ -392,7 +325,7 @@ public class PhoenixRuntime {
      * tenant-specific, the tenant ID column is not expected to have
      * been encoded and will not appear in the returned values.
      * @param conn an open connection
-     * @param fullTableName the full table name
+     * @param name the full table name
      * @param value the value that was encoded with {@link #encodePK(Connection, String, Object[])}
      * @return the Object values encoded in the byte array value
      * @throws SQLException
@@ -413,4 +346,181 @@ public class PhoenixRuntime {
         }
         return values;
     }
+
+    /**
+     * Represents the parsed commandline parameters definining the command or commands to be
+     * executed.
+     */
+    static class ExecutionCommand {
+        private String connectionString;
+        private List<String> columns;
+        private String tableName;
+        private char fieldDelimiter;
+        private char quoteCharacter;
+        private Character escapeCharacter;
+        private String arrayElementSeparator;
+        private boolean strict;
+        private List<String> inputFiles;
+
+        /**
+         * Factory method to build up an {@code ExecutionCommand} based on supplied parameters.
+         */
+        public static ExecutionCommand parseArgs(String[] args) {
+            Option tableOption = new Option("t", "table", true,
+                    "Overrides the table into which the CSV data is loaded and is case sensitive");
+            Option headerOption = new Option("h", "header", true, "Overrides the column names to" +
+                    " which the CSV data maps and is case sensitive. A special value of " +
+                    "in-line indicating that the first line of the CSV file determines the " +
+                    "column to which the data maps");
+            Option strictOption = new Option("s", "strict", false, "Use strict mode by throwing " +
+                    "an exception if a column name doesn't match during CSV loading");
+            Option delimiterOption = new Option("d", "delimiter", true,
+                    "Field delimiter for CSV loader. A digit is interpreted as " +
+                    "1 -> ctrl A, 2 -> ctrl B ... 9 -> ctrl I.");
+            Option quoteCharacterOption = new Option("q", "quote-character", true,
+                    "Quote character for CSV loader. A digit is interpreted as a control " +
+                            "character");
+            Option escapeCharacterOption = new Option("e", "escape-character", true,
+                    "Escape character for CSV loader. A digit is interpreted as a control " +
+                            "character");
+            Option arrayValueSeparatorOption = new Option("a", "array-separator", true,
+                    "Define the array element separator, defaults to ':'");
+            Options options = new Options();
+            options.addOption(tableOption);
+            options.addOption(headerOption);
+            options.addOption(strictOption);
+            options.addOption(delimiterOption);
+            options.addOption(quoteCharacterOption);
+            options.addOption(escapeCharacterOption);
+            options.addOption(arrayValueSeparatorOption);
+
+            CommandLineParser parser = new PosixParser();
+            CommandLine cmdLine = null;
+            try {
+                cmdLine = parser.parse(options, args);
+            } catch (ParseException e) {
+                usageError(options);
+            }
+
+            ExecutionCommand execCmd = new ExecutionCommand();
+
+            if (cmdLine.hasOption(tableOption.getOpt())) {
+                execCmd.tableName = cmdLine.getOptionValue(tableOption.getOpt());
+            }
+
+            if (cmdLine.hasOption(headerOption.getOpt())) {
+                String columnString = cmdLine.getOptionValue(headerOption.getOpt());
+                if (HEADER_IN_LINE.equals(columnString)) {
+                    execCmd.columns = ImmutableList.of();
+                } else {
+                    execCmd.columns = ImmutableList.copyOf(
+                            Splitter.on(",").trimResults().split(columnString));
+                }
+            }
+
+            execCmd.strict = cmdLine.hasOption(strictOption.getOpt());
+            execCmd.fieldDelimiter = getCharacter(
+                    cmdLine.getOptionValue(delimiterOption.getOpt(), ","));
+            execCmd.quoteCharacter = getCharacter(
+                    cmdLine.getOptionValue(quoteCharacterOption.getOpt(), "\""));
+
+            if (cmdLine.hasOption(escapeCharacterOption.getOpt())) {
+                execCmd.escapeCharacter = getCharacter(
+                        cmdLine.getOptionValue(escapeCharacterOption.getOpt(), "\\"));
+            }
+
+            execCmd.arrayElementSeparator = cmdLine.getOptionValue(
+                    arrayValueSeparatorOption.getOpt(),
+                    CSVCommonsLoader.DEFAULT_ARRAY_ELEMENT_SEPARATOR);
+
+
+            List<String> argList = Lists.newArrayList(cmdLine.getArgList());
+            if (argList.isEmpty()) {
+                usageError("Connection string to HBase must be supplied", options);
+            }
+            execCmd.connectionString = argList.remove(0);
+            List<String> inputFiles = Lists.newArrayList();
+            for (String arg : argList) {
+                if (arg.endsWith(CSV_FILE_EXT) || arg.endsWith(SQL_FILE_EXT)) {
+                    inputFiles.add(arg);
+                } else {
+                    usageError("Don't know how to interpret argument '" + arg + "'", options);
+                }
+            }
+
+            if (inputFiles.isEmpty()) {
+                usageError("At least one input file must be supplied", options);
+            }
+
+            execCmd.inputFiles = inputFiles;
+
+
+
+            return execCmd;
+        }
+
+        private static char getCharacter(String s) {
+            if (s.length() > 1) {
+                throw new IllegalArgumentException("Invalid single character: '" + s + "'");
+            }
+            return s.charAt(0);
+        }
+
+        private static void usageError(String errorMsg, Options options) {
+            System.out.println(errorMsg);
+            usageError(options);
+        }
+
+        private static void usageError(Options options) {
+            HelpFormatter formatter = new HelpFormatter();
+            formatter.printHelp(
+                    "psql [-t table-name] [-h comma-separated-column-names | in-line] [-d " +
+                            "field-delimiter-char quote-char escape-char]<zookeeper>  " +
+                            "<path-to-sql-or-csv-file>...",
+                    options);
+            System.out.println("Examples:\n" +
+                    "  psql localhost my_ddl.sql\n" +
+                    "  psql localhost my_ddl.sql my_table.csv\n" +
+                    "  psql -t MY_TABLE my_cluster:1825 my_table2012-Q3.csv\n" +
+                    "  psql -t MY_TABLE -h COL1,COL2,COL3 my_cluster:1825 my_table2012-Q3.csv\n" +
+                    "  psql -t MY_TABLE -h COL1,COL2,COL3 -d : my_cluster:1825 my_table2012-Q3.csv");
+            System.exit(-1);
+        }
+
+        public String getConnectionString() {
+            return connectionString;
+        }
+
+        public List<String> getColumns() {
+            return columns;
+        }
+
+        public String getTableName() {
+            return tableName;
+        }
+
+        public char getFieldDelimiter() {
+            return fieldDelimiter;
+        }
+
+        public char getQuoteCharacter() {
+            return quoteCharacter;
+        }
+
+        public Character getEscapeCharacter() {
+            return escapeCharacter;
+        }
+
+        public String getArrayElementSeparator() {
+            return arrayElementSeparator;
+        }
+
+        public List<String> getInputFiles() {
+            return inputFiles;
+        }
+
+        public boolean isStrict() {
+            return strict;
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/ec3e8400/phoenix-core/src/test/java/org/apache/phoenix/util/PhoenixRuntimeTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/PhoenixRuntimeTest.java b/phoenix-core/src/test/java/org/apache/phoenix/util/PhoenixRuntimeTest.java
index abb2b7a..a386093 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/util/PhoenixRuntimeTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/util/PhoenixRuntimeTest.java
@@ -19,12 +19,17 @@
 package org.apache.phoenix.util;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
 
+import java.io.File;
 import java.sql.Connection;
 import java.sql.Date;
 import java.sql.DriverManager;
 import java.util.Arrays;
 
+import com.google.common.collect.ImmutableList;
 import org.apache.phoenix.query.BaseConnectionlessQueryTest;
 import org.junit.Test;
 
@@ -41,4 +46,56 @@ public class PhoenixRuntimeTest extends BaseConnectionlessQueryTest {
         Object[] actualValues = PhoenixRuntime.decodePK(conn, "T", value);
         assertEquals(Arrays.asList(expectedValues), Arrays.asList(actualValues));
     }
+
+    @Test
+    public void testParseArguments_MinimalCase() {
+        PhoenixRuntime.ExecutionCommand execCmd = PhoenixRuntime.ExecutionCommand.parseArgs(
+                new String[] { "localhost", "test.csv" });
+
+
+        assertEquals(
+                "localhost",
+                execCmd.getConnectionString());
+
+        assertEquals(
+                ImmutableList.of("test.csv"),
+                execCmd.getInputFiles());
+
+        assertEquals(',', execCmd.getFieldDelimiter());
+        assertEquals('"', execCmd.getQuoteCharacter());
+        assertNull(execCmd.getEscapeCharacter());
+
+        assertNull(execCmd.getTableName());
+
+        assertNull(execCmd.getColumns());
+
+        assertFalse(execCmd.isStrict());
+
+        assertEquals(
+                CSVCommonsLoader.DEFAULT_ARRAY_ELEMENT_SEPARATOR,
+                execCmd.getArrayElementSeparator());
+    }
+
+    @Test
+    public void testParseArguments_FullOption() {
+        PhoenixRuntime.ExecutionCommand execCmd = PhoenixRuntime.ExecutionCommand.parseArgs(
+                new String[] { "-t", "mytable", "myzkhost:2181",  "--strict", "file1.sql",
+                        "test.csv", "file2.sql", "--header", "one, two,three", "-a", "!", "-d",
+                        ":", "-q", "3", "-e", "4" });
+
+        assertEquals("myzkhost:2181", execCmd.getConnectionString());
+
+        assertEquals(ImmutableList.of("file1.sql", "test.csv", "file2.sql"),
+                execCmd.getInputFiles());
+
+        assertEquals(':', execCmd.getFieldDelimiter());
+        assertEquals('3', execCmd.getQuoteCharacter());
+        assertEquals(Character.valueOf('4'), execCmd.getEscapeCharacter());
+
+        assertEquals("mytable", execCmd.getTableName());
+
+        assertEquals(ImmutableList.of("one", "two", "three"), execCmd.getColumns());
+        assertTrue(execCmd.isStrict());
+        assertEquals("!", execCmd.getArrayElementSeparator());
+    }
 }