You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by nd...@apache.org on 2015/04/17 01:29:39 UTC

[1/2] phoenix git commit: PHOENIX-971 Query server

Repository: phoenix
Updated Branches:
  refs/heads/master e25d7d098 -> e4f5cc627


http://git-wip-us.apache.org/repos/asf/phoenix/blob/e4f5cc62/phoenix-server/src/main/java/org/apache/phoenix/queryserver/server/PhoenixMetaFactoryImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-server/src/main/java/org/apache/phoenix/queryserver/server/PhoenixMetaFactoryImpl.java b/phoenix-server/src/main/java/org/apache/phoenix/queryserver/server/PhoenixMetaFactoryImpl.java
new file mode 100644
index 0000000..c74d2c9
--- /dev/null
+++ b/phoenix-server/src/main/java/org/apache/phoenix/queryserver/server/PhoenixMetaFactoryImpl.java
@@ -0,0 +1,76 @@
+/*
+ * 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.queryserver.server;
+
+import com.google.common.base.Preconditions;
+import org.apache.calcite.avatica.Meta;
+import org.apache.calcite.avatica.jdbc.JdbcMeta;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil;
+import org.apache.phoenix.util.QueryUtil;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+/**
+ * Bridge between Phoenix and Avatica.
+ */
+public class PhoenixMetaFactoryImpl extends Configured implements PhoenixMetaFactory {
+
+  // invoked via reflection
+  public PhoenixMetaFactoryImpl() {
+    super(HBaseConfiguration.create());
+  }
+
+  // invoked via reflection
+  public PhoenixMetaFactoryImpl(Configuration conf) {
+    super(conf);
+  }
+
+  @Override
+  public Meta create(List<String> args) {
+    Configuration conf = Preconditions.checkNotNull(getConf(), "Configuration must not be null.");
+    Properties info = new Properties();
+    info.putAll(conf.getValByRegex("avatica.*"));
+    try {
+      final String url;
+      if (args.size() == 0) {
+        url = QueryUtil.getConnectionUrl(info, conf);
+      } else if (args.size() == 1) {
+        url = args.get(0);
+      } else {
+        throw new RuntimeException(
+            "0 or 1 argument expected. Received " + Arrays.toString(args.toArray()));
+      }
+      // TODO: what about -D configs passed in from cli? How do they get pushed down?
+      return new JdbcMeta(url, info);
+    } catch (SQLException | ClassNotFoundException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e4f5cc62/phoenix-server/src/test/java/org/apache/phoenix/DriverCohabitationTest.java
----------------------------------------------------------------------
diff --git a/phoenix-server/src/test/java/org/apache/phoenix/DriverCohabitationTest.java b/phoenix-server/src/test/java/org/apache/phoenix/DriverCohabitationTest.java
new file mode 100644
index 0000000..1df6d2c
--- /dev/null
+++ b/phoenix-server/src/test/java/org/apache/phoenix/DriverCohabitationTest.java
@@ -0,0 +1,65 @@
+/*
+ * 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;
+
+import org.apache.phoenix.queryserver.client.ThinClientUtil;
+import org.apache.phoenix.util.QueryUtil;
+import org.junit.Test;
+
+import java.sql.Driver;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.Collections;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Ensure the "thick" Phoenix driver and it's "thin" counterpart can coexist on
+ * the same classpath.
+ */
+public class DriverCohabitationTest {
+
+  @Test
+  public void testDriverCohabitation() throws SQLException {
+    Driver thickDriver = null;
+    Driver thinDriver = null;
+
+    for (Driver d : Collections.list(DriverManager.getDrivers())) {
+      if (d instanceof org.apache.phoenix.jdbc.PhoenixDriver) {
+        thickDriver = d;
+      } else if (d instanceof org.apache.phoenix.queryserver.client.Driver) {
+        thinDriver = d;
+      }
+    }
+    assertNotNull("Thick driver not registered with DriverManager.", thickDriver);
+    assertNotNull("Thin driver not registered with DriverManager.", thinDriver);
+
+    final String thickUrl = QueryUtil.getUrl("localhost");
+    final String thinUrl = ThinClientUtil.getConnectionUrl("localhost", 1234);
+    assertTrue("Thick driver should accept connections like " + thickUrl,
+        thickDriver.acceptsURL(thickUrl));
+    assertFalse("Thick driver should reject connections like " + thinUrl,
+        thickDriver.acceptsURL(thinUrl));
+    assertTrue("Thin driver should accept connections like " + thinUrl,
+        thinDriver.acceptsURL(thinUrl));
+    assertFalse("Thin driver should reject connections like " + thickUrl,
+        thinDriver.acceptsURL(thickUrl));
+  }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e4f5cc62/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 977218d..d38f2e4 100644
--- a/pom.xml
+++ b/pom.xml
@@ -26,6 +26,8 @@
     <module>phoenix-core</module>
     <module>phoenix-flume</module>
     <module>phoenix-pig</module>
+    <module>phoenix-server-client</module>
+    <module>phoenix-server</module>
     <module>phoenix-assembly</module>
     <module>phoenix-pherf</module>
     <module>phoenix-spark</module>
@@ -91,7 +93,7 @@
     <commons-configuration.version>1.6</commons-configuration.version>
     <commons-io.version>2.1</commons-io.version>
     <commons-lang.version>2.5</commons-lang.version>
-    <commons-logging.version>1.1.1</commons-logging.version>
+    <commons-logging.version>1.2</commons-logging.version>
     <commons-csv.version>1.0</commons-csv.version>
     <sqlline.version>1.1.8</sqlline.version>
     <guava.version>12.0.1</guava.version>
@@ -106,6 +108,7 @@
     <collections.version>3.2.1</collections.version>
     <jodatime.version>2.7</jodatime.version>
     <joni.version>2.1.2</joni.version>
+    <calcite.version>1.2.0-incubating</calcite.version>
 
     <!-- Test Dependencies -->
     <mockito-all.version>1.8.5</mockito-all.version>
@@ -161,7 +164,7 @@
                     </goals>
                   </pluginExecutionFilter>
                   <action>
-                    <ignore></ignore>
+                    <ignore />
                   </action>
                 </pluginExecution>
               </pluginExecutions>
@@ -205,7 +208,7 @@
               <exclude>.project</exclude>
               <exclude>.classpath</exclude>
               <exclude>.settings/**</exclude>
-              <exclude>**/resources/java.sql.Driver</exclude>
+              <exclude>**/java.sql.Driver</exclude>
               <!-- exclude protobuf files -->
               <exclude>**/generated/**</exclude>
             </excludes>
@@ -433,6 +436,16 @@
         <artifactId>phoenix-spark</artifactId>
         <version>${project.version}</version>
       </dependency>
+      <dependency>
+        <groupId>org.apache.phoenix</groupId>
+        <artifactId>phoenix-server</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.phoenix</groupId>
+        <artifactId>phoenix-server-client</artifactId>
+        <version>${project.version}</version>
+      </dependency>
 
       <!-- HBase dependencies -->
       <dependency>
@@ -450,6 +463,19 @@
       </dependency>
       <dependency>
         <groupId>org.apache.hbase</groupId>
+        <artifactId>hbase-it</artifactId>
+        <version>${hbase.version}</version>
+        <type>test-jar</type>
+        <scope>test</scope>
+        <exclusions>
+          <exclusion>
+            <groupId>org.jruby</groupId>
+            <artifactId>jruby-complete</artifactId>
+          </exclusion>
+        </exclusions>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hbase</groupId>
         <artifactId>hbase-protocol</artifactId>
         <version>${hbase.version}</version>
       </dependency>
@@ -524,6 +550,16 @@
           </exclusion>
         </exclusions>
       </dependency>
+      <dependency>
+        <groupId>org.apache.calcite</groupId>
+        <artifactId>calcite-avatica</artifactId>
+        <version>${calcite.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.calcite</groupId>
+        <artifactId>calcite-avatica-server</artifactId>
+        <version>${calcite.version}</version>
+      </dependency>
 
       <!-- Make sure we have all the antlr dependencies -->
       <dependency>
@@ -633,6 +669,11 @@
         <version>${slf4j.version}</version>
       </dependency>
       <dependency>
+        <groupId>commons-logging</groupId>
+        <artifactId>commons-logging</artifactId>
+        <version>${commons-logging.version}</version>
+      </dependency>
+      <dependency>
         <groupId>org.apache.htrace</groupId>
         <artifactId>htrace-core</artifactId>
         <version>${htrace.version}</version>


[2/2] phoenix git commit: PHOENIX-971 Query server

Posted by nd...@apache.org.
PHOENIX-971 Query server


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

Branch: refs/heads/master
Commit: e4f5cc6274db747209b68ada15a3030754e90d9c
Parents: e25d7d0
Author: Nick Dimiduk <nd...@apache.org>
Authored: Thu Apr 16 16:11:36 2015 -0700
Committer: Nick Dimiduk <nd...@apache.org>
Committed: Thu Apr 16 16:11:36 2015 -0700

----------------------------------------------------------------------
 NOTICE                                          |  10 +
 bin/daemon.py                                   | 989 +++++++++++++++++++
 bin/log4j.properties                            |   9 +-
 bin/phoenix_utils.py                            |   4 +
 bin/queryserver.py                              | 147 +++
 bin/sqlline-thin.py                             |  89 ++
 dev/make_rc.sh                                  |   5 +
 phoenix-assembly/pom.xml                        |   8 +
 .../src/build/components-minimal.xml            |   2 +
 .../apache/phoenix/jdbc/PhoenixConnection.java  |   5 +-
 .../phoenix/jdbc/PhoenixEmbeddedDriver.java     |   5 +
 .../org/apache/phoenix/util/PhoenixRuntime.java |   5 +
 phoenix-server-client/pom.xml                   |  62 ++
 .../src/build/query-server-thin-client.xml      |  40 +
 .../phoenix/queryserver/client/Driver.java      |  49 +
 .../queryserver/client/ThinClientUtil.java      |  35 +
 .../resources/META-INF/services/java.sql.Driver |   1 +
 .../org-apache-phoenix-remote-jdbc.properties   |  25 +
 phoenix-server/pom.xml                          | 107 ++
 .../src/build/query-server-runnable.xml         |  39 +
 .../phoenix/end2end/QueryServerBasicsIT.java    | 150 +++
 .../phoenix/end2end/QueryServerThread.java      |  51 +
 .../apache/phoenix/queryserver/server/Main.java | 158 +++
 .../queryserver/server/PhoenixMetaFactory.java  |  28 +
 .../server/PhoenixMetaFactoryImpl.java          |  76 ++
 .../apache/phoenix/DriverCohabitationTest.java  |  65 ++
 pom.xml                                         |  47 +-
 27 files changed, 2205 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/e4f5cc62/NOTICE
----------------------------------------------------------------------
diff --git a/NOTICE b/NOTICE
index 0bd2251..5418b0c 100644
--- a/NOTICE
+++ b/NOTICE
@@ -28,3 +28,13 @@ The phoenix-spark module has been adapted from the phoenix-spark library
 distributed under the terms of the Apache 2 license. Original source copyright:
 Copyright 2014 Simply Measured, Inc.
 Copyright 2015 Interset Software Inc.
+
+The file bin/daemon.py is based on the file of the same name in python-daemon 2.0.5
+(https://pypi.python.org/pypi/python-daemon/) distributed under the terms of
+the Apache 2 license. Original source copyright:
+# Copyright © 2008–2015 Ben Finney <be...@benfinney.id.au>
+# Copyright © 2007–2008 Robert Niederreiter, Jens Klein
+# Copyright © 2004–2005 Chad J. Schroeder
+# Copyright © 2003 Clark Evans
+# Copyright © 2002 Noah Spurrier
+# Copyright © 2001 Jürgen Hermann

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e4f5cc62/bin/daemon.py
----------------------------------------------------------------------
diff --git a/bin/daemon.py b/bin/daemon.py
new file mode 100644
index 0000000..aeebae3
--- /dev/null
+++ b/bin/daemon.py
@@ -0,0 +1,989 @@
+#!/usr/bin/env python
+# -*- coding: utf-8 -*-
+############################################################################
+#
+# 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.
+#
+############################################################################
+
+# daemon/daemon.py
+# Part of ‘python-daemon’, an implementation of PEP 3143.
+#
+# Copyright © 2008–2015 Ben Finney <be...@benfinney.id.au>
+# Copyright © 2007–2008 Robert Niederreiter, Jens Klein
+# Copyright © 2004–2005 Chad J. Schroeder
+# Copyright © 2003 Clark Evans
+# Copyright © 2002 Noah Spurrier
+# Copyright © 2001 Jürgen Hermann
+#
+# This is free software: you may copy, modify, and/or distribute this work
+# under the terms of the Apache License, version 2.0 as published by the
+# Apache Software Foundation.
+# No warranty expressed or implied. See the file ‘LICENSE.ASF-2’ for details.
+
+#
+# Apache Phoenix note: this file is `daemon.py` from the package
+# `python-daemon 2.0.5`, https://pypi.python.org/pypi/python-daemon/
+#
+# The class `PidFile` was added for adapting the `lockfile` package's interface
+# without depending on yet another 3rd party package. Based on example from
+# http://code.activestate.com/recipes/577911-context-manager-for-a-daemon-pid-file/
+#
+
+""" Daemon process behaviour.
+    """
+
+from __future__ import (absolute_import, unicode_literals)
+
+import os
+import sys
+import resource
+import errno
+import signal
+import socket
+import atexit
+import fcntl
+try:
+    # Python 2 has both ‘str’ (bytes) and ‘unicode’ (text).
+    basestring = basestring
+    unicode = unicode
+except NameError:
+    # Python 3 names the Unicode data type ‘str’.
+    basestring = str
+    unicode = str
+
+
+class DaemonError(Exception):
+    """ Base exception class for errors from this module. """
+
+    def __init__(self, *args, **kwargs):
+        self._chain_from_context()
+
+        super(DaemonError, self).__init__(*args, **kwargs)
+
+    def _chain_from_context(self):
+        _chain_exception_from_existing_exception_context(self, as_cause=True)
+
+
+class DaemonOSEnvironmentError(DaemonError, OSError):
+    """ Exception raised when daemon OS environment setup receives error. """
+
+
+class DaemonProcessDetachError(DaemonError, OSError):
+    """ Exception raised when process detach fails. """
+
+
+class DaemonContext:
+    """ Context for turning the current program into a daemon process.
+
+        A `DaemonContext` instance represents the behaviour settings and
+        process context for the program when it becomes a daemon. The
+        behaviour and environment is customised by setting options on the
+        instance, before calling the `open` method.
+
+        Each option can be passed as a keyword argument to the `DaemonContext`
+        constructor, or subsequently altered by assigning to an attribute on
+        the instance at any time prior to calling `open`. That is, for
+        options named `wibble` and `wubble`, the following invocation::
+
+            foo = daemon.DaemonContext(wibble=bar, wubble=baz)
+            foo.open()
+
+        is equivalent to::
+
+            foo = daemon.DaemonContext()
+            foo.wibble = bar
+            foo.wubble = baz
+            foo.open()
+
+        The following options are defined.
+
+        `files_preserve`
+            :Default: ``None``
+
+            List of files that should *not* be closed when starting the
+            daemon. If ``None``, all open file descriptors will be closed.
+
+            Elements of the list are file descriptors (as returned by a file
+            object's `fileno()` method) or Python `file` objects. Each
+            specifies a file that is not to be closed during daemon start.
+
+        `chroot_directory`
+            :Default: ``None``
+
+            Full path to a directory to set as the effective root directory of
+            the process. If ``None``, specifies that the root directory is not
+            to be changed.
+
+        `working_directory`
+            :Default: ``'/'``
+
+            Full path of the working directory to which the process should
+            change on daemon start.
+
+            Since a filesystem cannot be unmounted if a process has its
+            current working directory on that filesystem, this should either
+            be left at default or set to a directory that is a sensible “home
+            directory” for the daemon while it is running.
+
+        `umask`
+            :Default: ``0``
+
+            File access creation mask (“umask”) to set for the process on
+            daemon start.
+
+            A daemon should not rely on the parent process's umask value,
+            which is beyond its control and may prevent creating a file with
+            the required access mode. So when the daemon context opens, the
+            umask is set to an explicit known value.
+
+            If the conventional value of 0 is too open, consider setting a
+            value such as 0o022, 0o027, 0o077, or another specific value.
+            Otherwise, ensure the daemon creates every file with an
+            explicit access mode for the purpose.
+
+        `pidfile`
+            :Default: ``None``
+
+            Context manager for a PID lock file. When the daemon context opens
+            and closes, it enters and exits the `pidfile` context manager.
+
+        `detach_process`
+            :Default: ``None``
+
+            If ``True``, detach the process context when opening the daemon
+            context; if ``False``, do not detach.
+
+            If unspecified (``None``) during initialisation of the instance,
+            this will be set to ``True`` by default, and ``False`` only if
+            detaching the process is determined to be redundant; for example,
+            in the case when the process was started by `init`, by `initd`, or
+            by `inetd`.
+
+        `signal_map`
+            :Default: system-dependent
+
+            Mapping from operating system signals to callback actions.
+
+            The mapping is used when the daemon context opens, and determines
+            the action for each signal's signal handler:
+
+            * A value of ``None`` will ignore the signal (by setting the
+              signal action to ``signal.SIG_IGN``).
+
+            * A string value will be used as the name of an attribute on the
+              ``DaemonContext`` instance. The attribute's value will be used
+              as the action for the signal handler.
+
+            * Any other value will be used as the action for the
+              signal handler. See the ``signal.signal`` documentation
+              for details of the signal handler interface.
+
+            The default value depends on which signals are defined on the
+            running system. Each item from the list below whose signal is
+            actually defined in the ``signal`` module will appear in the
+            default map:
+
+            * ``signal.SIGTTIN``: ``None``
+
+            * ``signal.SIGTTOU``: ``None``
+
+            * ``signal.SIGTSTP``: ``None``
+
+            * ``signal.SIGTERM``: ``'terminate'``
+
+            Depending on how the program will interact with its child
+            processes, it may need to specify a signal map that
+            includes the ``signal.SIGCHLD`` signal (received when a
+            child process exits). See the specific operating system's
+            documentation for more detail on how to determine what
+            circumstances dictate the need for signal handlers.
+
+        `uid`
+            :Default: ``os.getuid()``
+
+        `gid`
+            :Default: ``os.getgid()``
+
+            The user ID (“UID”) value and group ID (“GID”) value to switch
+            the process to on daemon start.
+
+            The default values, the real UID and GID of the process, will
+            relinquish any effective privilege elevation inherited by the
+            process.
+
+        `prevent_core`
+            :Default: ``True``
+
+            If true, prevents the generation of core files, in order to avoid
+            leaking sensitive information from daemons run as `root`.
+
+        `stdin`
+            :Default: ``None``
+
+        `stdout`
+            :Default: ``None``
+
+        `stderr`
+            :Default: ``None``
+
+            Each of `stdin`, `stdout`, and `stderr` is a file-like object
+            which will be used as the new file for the standard I/O stream
+            `sys.stdin`, `sys.stdout`, and `sys.stderr` respectively. The file
+            should therefore be open, with a minimum of mode 'r' in the case
+            of `stdin`, and mimimum of mode 'w+' in the case of `stdout` and
+            `stderr`.
+
+            If the object has a `fileno()` method that returns a file
+            descriptor, the corresponding file will be excluded from being
+            closed during daemon start (that is, it will be treated as though
+            it were listed in `files_preserve`).
+
+            If ``None``, the corresponding system stream is re-bound to the
+            file named by `os.devnull`.
+
+        """
+
+    __metaclass__ = type
+
+    def __init__(
+            self,
+            chroot_directory=None,
+            working_directory="/",
+            umask=0,
+            uid=None,
+            gid=None,
+            prevent_core=True,
+            detach_process=None,
+            files_preserve=None,
+            pidfile=None,
+            stdin=None,
+            stdout=None,
+            stderr=None,
+            signal_map=None,
+            ):
+        """ Set up a new instance. """
+        self.chroot_directory = chroot_directory
+        self.working_directory = working_directory
+        self.umask = umask
+        self.prevent_core = prevent_core
+        self.files_preserve = files_preserve
+        self.pidfile = pidfile
+        self.stdin = stdin
+        self.stdout = stdout
+        self.stderr = stderr
+
+        if uid is None:
+            uid = os.getuid()
+        self.uid = uid
+        if gid is None:
+            gid = os.getgid()
+        self.gid = gid
+
+        if detach_process is None:
+            detach_process = is_detach_process_context_required()
+        self.detach_process = detach_process
+
+        if signal_map is None:
+            signal_map = make_default_signal_map()
+        self.signal_map = signal_map
+
+        self._is_open = False
+
+    @property
+    def is_open(self):
+        """ ``True`` if the instance is currently open. """
+        return self._is_open
+
+    def open(self):
+        """ Become a daemon process.
+
+            :return: ``None``.
+
+            Open the daemon context, turning the current program into a daemon
+            process. This performs the following steps:
+
+            * If this instance's `is_open` property is true, return
+              immediately. This makes it safe to call `open` multiple times on
+              an instance.
+
+            * If the `prevent_core` attribute is true, set the resource limits
+              for the process to prevent any core dump from the process.
+
+            * If the `chroot_directory` attribute is not ``None``, set the
+              effective root directory of the process to that directory (via
+              `os.chroot`).
+
+              This allows running the daemon process inside a “chroot gaol”
+              as a means of limiting the system's exposure to rogue behaviour
+              by the process. Note that the specified directory needs to
+              already be set up for this purpose.
+
+            * Set the process UID and GID to the `uid` and `gid` attribute
+              values.
+
+            * Close all open file descriptors. This excludes those listed in
+              the `files_preserve` attribute, and those that correspond to the
+              `stdin`, `stdout`, or `stderr` attributes.
+
+            * Change current working directory to the path specified by the
+              `working_directory` attribute.
+
+            * Reset the file access creation mask to the value specified by
+              the `umask` attribute.
+
+            * If the `detach_process` option is true, detach the current
+              process into its own process group, and disassociate from any
+              controlling terminal.
+
+            * Set signal handlers as specified by the `signal_map` attribute.
+
+            * If any of the attributes `stdin`, `stdout`, `stderr` are not
+              ``None``, bind the system streams `sys.stdin`, `sys.stdout`,
+              and/or `sys.stderr` to the files represented by the
+              corresponding attributes. Where the attribute has a file
+              descriptor, the descriptor is duplicated (instead of re-binding
+              the name).
+
+            * If the `pidfile` attribute is not ``None``, enter its context
+              manager.
+
+            * Mark this instance as open (for the purpose of future `open` and
+              `close` calls).
+
+            * Register the `close` method to be called during Python's exit
+              processing.
+
+            When the function returns, the running program is a daemon
+            process.
+
+            """
+        if self.is_open:
+            return
+
+        if self.chroot_directory is not None:
+            change_root_directory(self.chroot_directory)
+
+        if self.prevent_core:
+            prevent_core_dump()
+
+        change_file_creation_mask(self.umask)
+        change_working_directory(self.working_directory)
+        change_process_owner(self.uid, self.gid)
+
+        if self.detach_process:
+            detach_process_context()
+
+        signal_handler_map = self._make_signal_handler_map()
+        set_signal_handlers(signal_handler_map)
+
+        exclude_fds = self._get_exclude_file_descriptors()
+        close_all_open_files(exclude=exclude_fds)
+
+        redirect_stream(sys.stdin, self.stdin)
+        redirect_stream(sys.stdout, self.stdout)
+        redirect_stream(sys.stderr, self.stderr)
+
+        if self.pidfile is not None:
+            self.pidfile.__enter__()
+
+        self._is_open = True
+
+        register_atexit_function(self.close)
+
+    def __enter__(self):
+        """ Context manager entry point. """
+        self.open()
+        return self
+
+    def close(self):
+        """ Exit the daemon process context.
+
+            :return: ``None``.
+
+            Close the daemon context. This performs the following steps:
+
+            * If this instance's `is_open` property is false, return
+              immediately. This makes it safe to call `close` multiple times
+              on an instance.
+
+            * If the `pidfile` attribute is not ``None``, exit its context
+              manager.
+
+            * Mark this instance as closed (for the purpose of future `open`
+              and `close` calls).
+
+            """
+        if not self.is_open:
+            return
+
+        if self.pidfile is not None:
+            # Follow the interface for telling a context manager to exit,
+            # <URL:http://docs.python.org/library/stdtypes.html#typecontextmanager>.
+            self.pidfile.__exit__(None, None, None)
+
+        self._is_open = False
+
+    def __exit__(self, exc_type, exc_value, traceback):
+        """ Context manager exit point. """
+        self.close()
+
+    def terminate(self, signal_number, stack_frame):
+        """ Signal handler for end-process signals.
+
+            :param signal_number: The OS signal number received.
+            :param stack_frame: The frame object at the point the
+                signal was received.
+            :return: ``None``.
+
+            Signal handler for the ``signal.SIGTERM`` signal. Performs the
+            following step:
+
+            * Raise a ``SystemExit`` exception explaining the signal.
+
+            """
+        exception = SystemExit(
+                "Terminating on signal {signal_number!r}".format(
+                    signal_number=signal_number))
+        raise exception
+
+    def _get_exclude_file_descriptors(self):
+        """ Get the set of file descriptors to exclude closing.
+
+            :return: A set containing the file descriptors for the
+                files to be preserved.
+
+            The file descriptors to be preserved are those from the
+            items in `files_preserve`, and also each of `stdin`,
+            `stdout`, and `stderr`. For each item:
+
+            * If the item is ``None``, it is omitted from the return
+              set.
+
+            * If the item's ``fileno()`` method returns a value, that
+              value is in the return set.
+
+            * Otherwise, the item is in the return set verbatim.
+
+            """
+        files_preserve = self.files_preserve
+        if files_preserve is None:
+            files_preserve = []
+        files_preserve.extend(
+                item for item in [self.stdin, self.stdout, self.stderr]
+                if hasattr(item, 'fileno'))
+
+        exclude_descriptors = set()
+        for item in files_preserve:
+            if item is None:
+                continue
+            file_descriptor = _get_file_descriptor(item)
+            if file_descriptor is not None:
+                exclude_descriptors.add(file_descriptor)
+            else:
+                exclude_descriptors.add(item)
+
+        return exclude_descriptors
+
+    def _make_signal_handler(self, target):
+        """ Make the signal handler for a specified target object.
+
+            :param target: A specification of the target for the
+                handler; see below.
+            :return: The value for use by `signal.signal()`.
+
+            If `target` is ``None``, return ``signal.SIG_IGN``. If `target`
+            is a text string, return the attribute of this instance named
+            by that string. Otherwise, return `target` itself.
+
+            """
+        if target is None:
+            result = signal.SIG_IGN
+        elif isinstance(target, unicode):
+            name = target
+            result = getattr(self, name)
+        else:
+            result = target
+
+        return result
+
+    def _make_signal_handler_map(self):
+        """ Make the map from signals to handlers for this instance.
+
+            :return: The constructed signal map for this instance.
+
+            Construct a map from signal numbers to handlers for this
+            context instance, suitable for passing to
+            `set_signal_handlers`.
+
+            """
+        signal_handler_map = dict(
+                (signal_number, self._make_signal_handler(target))
+                for (signal_number, target) in self.signal_map.items())
+        return signal_handler_map
+
+
+def _get_file_descriptor(obj):
+    """ Get the file descriptor, if the object has one.
+
+        :param obj: The object expected to be a file-like object.
+        :return: The file descriptor iff the file supports it; otherwise
+            ``None``.
+
+        The object may be a non-file object. It may also be a
+        file-like object with no support for a file descriptor. In
+        either case, return ``None``.
+
+        """
+    file_descriptor = None
+    if hasattr(obj, 'fileno'):
+        try:
+            file_descriptor = obj.fileno()
+        except ValueError:
+            # The item doesn't support a file descriptor.
+            pass
+
+    return file_descriptor
+
+
+def change_working_directory(directory):
+    """ Change the working directory of this process.
+
+        :param directory: The target directory path.
+        :return: ``None``.
+
+        """
+    try:
+        os.chdir(directory)
+    except Exception as exc:
+        error = DaemonOSEnvironmentError(
+                "Unable to change working directory ({exc})".format(exc=exc))
+        raise error
+
+
+def change_root_directory(directory):
+    """ Change the root directory of this process.
+
+        :param directory: The target directory path.
+        :return: ``None``.
+
+        Set the current working directory, then the process root directory,
+        to the specified `directory`. Requires appropriate OS privileges
+        for this process.
+
+        """
+    try:
+        os.chdir(directory)
+        os.chroot(directory)
+    except Exception as exc:
+        error = DaemonOSEnvironmentError(
+                "Unable to change root directory ({exc})".format(exc=exc))
+        raise error
+
+
+def change_file_creation_mask(mask):
+    """ Change the file creation mask for this process.
+
+        :param mask: The numeric file creation mask to set.
+        :return: ``None``.
+
+        """
+    try:
+        os.umask(mask)
+    except Exception as exc:
+        error = DaemonOSEnvironmentError(
+                "Unable to change file creation mask ({exc})".format(exc=exc))
+        raise error
+
+
+def change_process_owner(uid, gid):
+    """ Change the owning UID and GID of this process.
+
+        :param uid: The target UID for the daemon process.
+        :param gid: The target GID for the daemon process.
+        :return: ``None``.
+
+        Set the GID then the UID of the process (in that order, to avoid
+        permission errors) to the specified `gid` and `uid` values.
+        Requires appropriate OS privileges for this process.
+
+        """
+    try:
+        os.setgid(gid)
+        os.setuid(uid)
+    except Exception as exc:
+        error = DaemonOSEnvironmentError(
+                "Unable to change process owner ({exc})".format(exc=exc))
+        raise error
+
+
+def prevent_core_dump():
+    """ Prevent this process from generating a core dump.
+
+        :return: ``None``.
+
+        Set the soft and hard limits for core dump size to zero. On Unix,
+        this entirely prevents the process from creating core dump.
+
+        """
+    core_resource = resource.RLIMIT_CORE
+
+    try:
+        # Ensure the resource limit exists on this platform, by requesting
+        # its current value.
+        core_limit_prev = resource.getrlimit(core_resource)
+    except ValueError as exc:
+        error = DaemonOSEnvironmentError(
+                "System does not support RLIMIT_CORE resource limit"
+                " ({exc})".format(exc=exc))
+        raise error
+
+    # Set hard and soft limits to zero, i.e. no core dump at all.
+    core_limit = (0, 0)
+    resource.setrlimit(core_resource, core_limit)
+
+
+def detach_process_context():
+    """ Detach the process context from parent and session.
+
+        :return: ``None``.
+
+        Detach from the parent process and session group, allowing the
+        parent to exit while this process continues running.
+
+        Reference: “Advanced Programming in the Unix Environment”,
+        section 13.3, by W. Richard Stevens, published 1993 by
+        Addison-Wesley.
+
+        """
+
+    def fork_then_exit_parent(error_message):
+        """ Fork a child process, then exit the parent process.
+
+            :param error_message: Message for the exception in case of a
+                detach failure.
+            :return: ``None``.
+            :raise DaemonProcessDetachError: If the fork fails.
+
+            """
+        try:
+            pid = os.fork()
+            if pid > 0:
+                os._exit(0)
+        except OSError as exc:
+            error = DaemonProcessDetachError(
+                    "{message}: [{exc.errno:d}] {exc.strerror}".format(
+                        message=error_message, exc=exc))
+            raise error
+
+    fork_then_exit_parent(error_message="Failed first fork")
+    os.setsid()
+    fork_then_exit_parent(error_message="Failed second fork")
+
+
+def is_process_started_by_init():
+    """ Determine whether the current process is started by `init`.
+
+        :return: ``True`` iff the parent process is `init`; otherwise
+            ``False``.
+
+        The `init` process is the one with process ID of 1.
+
+        """
+    result = False
+
+    init_pid = 1
+    if os.getppid() == init_pid:
+        result = True
+
+    return result
+
+
+def is_socket(fd):
+    """ Determine whether the file descriptor is a socket.
+
+        :param fd: The file descriptor to interrogate.
+        :return: ``True`` iff the file descriptor is a socket; otherwise
+            ``False``.
+
+        Query the socket type of `fd`. If there is no error, the file is a
+        socket.
+
+        """
+    result = False
+
+    file_socket = socket.fromfd(fd, socket.AF_INET, socket.SOCK_RAW)
+
+    try:
+        socket_type = file_socket.getsockopt(
+                socket.SOL_SOCKET, socket.SO_TYPE)
+    except socket.error as exc:
+        exc_errno = exc.args[0]
+        if exc_errno == errno.ENOTSOCK:
+            # Socket operation on non-socket.
+            pass
+        else:
+            # Some other socket error.
+            result = True
+    else:
+        # No error getting socket type.
+        result = True
+
+    return result
+
+
+def is_process_started_by_superserver():
+    """ Determine whether the current process is started by the superserver.
+
+        :return: ``True`` if this process was started by the internet
+            superserver; otherwise ``False``.
+
+        The internet superserver creates a network socket, and
+        attaches it to the standard streams of the child process. If
+        that is the case for this process, return ``True``, otherwise
+        ``False``.
+
+        """
+    result = False
+
+    stdin_fd = sys.__stdin__.fileno()
+    if is_socket(stdin_fd):
+        result = True
+
+    return result
+
+
+def is_detach_process_context_required():
+    """ Determine whether detaching the process context is required.
+
+        :return: ``True`` iff the process is already detached; otherwise
+            ``False``.
+
+        The process environment is interrogated for the following:
+
+        * Process was started by `init`; or
+
+        * Process was started by `inetd`.
+
+        If any of the above are true, the process is deemed to be already
+        detached.
+
+        """
+    result = True
+    if is_process_started_by_init() or is_process_started_by_superserver():
+        result = False
+
+    return result
+
+
+def close_file_descriptor_if_open(fd):
+    """ Close a file descriptor if already open.
+
+        :param fd: The file descriptor to close.
+        :return: ``None``.
+
+        Close the file descriptor `fd`, suppressing an error in the
+        case the file was not open.
+
+        """
+    try:
+        os.close(fd)
+    except EnvironmentError as exc:
+        if exc.errno == errno.EBADF:
+            # File descriptor was not open.
+            pass
+        else:
+            error = DaemonOSEnvironmentError(
+                    "Failed to close file descriptor {fd:d} ({exc})".format(
+                        fd=fd, exc=exc))
+            raise error
+
+
+MAXFD = 2048
+
+def get_maximum_file_descriptors():
+    """ Get the maximum number of open file descriptors for this process.
+
+        :return: The number (integer) to use as the maximum number of open
+            files for this process.
+
+        The maximum is the process hard resource limit of maximum number of
+        open file descriptors. If the limit is “infinity”, a default value
+        of ``MAXFD`` is returned.
+
+        """
+    limits = resource.getrlimit(resource.RLIMIT_NOFILE)
+    result = limits[1]
+    if result == resource.RLIM_INFINITY:
+        result = MAXFD
+    return result
+
+
+def close_all_open_files(exclude=set()):
+    """ Close all open file descriptors.
+
+        :param exclude: Collection of file descriptors to skip when closing
+            files.
+        :return: ``None``.
+
+        Closes every file descriptor (if open) of this process. If
+        specified, `exclude` is a set of file descriptors to *not*
+        close.
+
+        """
+    maxfd = get_maximum_file_descriptors()
+    for fd in reversed(range(maxfd)):
+        if fd not in exclude:
+            close_file_descriptor_if_open(fd)
+
+
+def redirect_stream(system_stream, target_stream):
+    """ Redirect a system stream to a specified file.
+
+        :param standard_stream: A file object representing a standard I/O
+            stream.
+        :param target_stream: The target file object for the redirected
+            stream, or ``None`` to specify the null device.
+        :return: ``None``.
+
+        `system_stream` is a standard system stream such as
+        ``sys.stdout``. `target_stream` is an open file object that
+        should replace the corresponding system stream object.
+
+        If `target_stream` is ``None``, defaults to opening the
+        operating system's null device and using its file descriptor.
+
+        """
+    if target_stream is None:
+        target_fd = os.open(os.devnull, os.O_RDWR)
+    else:
+        target_fd = target_stream.fileno()
+    os.dup2(target_fd, system_stream.fileno())
+
+
+def make_default_signal_map():
+    """ Make the default signal map for this system.
+
+        :return: A mapping from signal number to handler object.
+
+        The signals available differ by system. The map will not contain
+        any signals not defined on the running system.
+
+        """
+    name_map = {
+            'SIGTSTP': None,
+            'SIGTTIN': None,
+            'SIGTTOU': None,
+            'SIGTERM': 'terminate',
+            }
+    signal_map = dict(
+            (getattr(signal, name), target)
+            for (name, target) in name_map.items()
+            if hasattr(signal, name))
+
+    return signal_map
+
+
+def set_signal_handlers(signal_handler_map):
+    """ Set the signal handlers as specified.
+
+        :param signal_handler_map: A map from signal number to handler
+            object.
+        :return: ``None``.
+
+        See the `signal` module for details on signal numbers and signal
+        handlers.
+
+        """
+    for (signal_number, handler) in signal_handler_map.items():
+        signal.signal(signal_number, handler)
+
+
+def register_atexit_function(func):
+    """ Register a function for processing at program exit.
+
+        :param func: A callable function expecting no arguments.
+        :return: ``None``.
+
+        The function `func` is registered for a call with no arguments
+        at program exit.
+
+        """
+    atexit.register(func)
+
+
+def _chain_exception_from_existing_exception_context(exc, as_cause=False):
+    """ Decorate the specified exception with the existing exception context.
+
+        :param exc: The exception instance to decorate.
+        :param as_cause: If true, the existing context is declared to be
+            the cause of the exception.
+        :return: ``None``.
+
+        :PEP:`344` describes syntax and attributes (`__traceback__`,
+        `__context__`, `__cause__`) for use in exception chaining.
+
+        Python 2 does not have that syntax, so this function decorates
+        the exception with values from the current exception context.
+
+        """
+    (existing_exc_type, existing_exc, existing_traceback) = sys.exc_info()
+    if as_cause:
+        exc.__cause__ = existing_exc
+    else:
+        exc.__context__ = existing_exc
+    exc.__traceback__ = existing_traceback
+
+class PidFile(object):
+    """
+Adapter between a file path string and the `lockfile` API [0]. Based example
+found at [1].
+
+[0]: https://pythonhosted.org/lockfile/lockfile.html
+[1]: http://code.activestate.com/recipes/577911-context-manager-for-a-daemon-pid-file/
+"""
+    def __init__(self, path, enter_err_msg=None):
+        self.path = path
+        self.enter_err_msg = enter_err_msg
+        self.pidfile = None
+
+    def __enter__(self):
+        self.pidfile = open(self.path, 'a+')
+        try:
+            fcntl.flock(self.pidfile.fileno(), fcntl.LOCK_EX | fcntl.LOCK_NB)
+        except IOError:
+            sys.exit(self.enter_err_msg)
+        self.pidfile.seek(0)
+        self.pidfile.truncate()
+        self.pidfile.write(str(os.getpid()))
+        self.pidfile.flush()
+        self.pidfile.seek(0)
+        return self.pidfile
+
+    def __exit__(self, exc_type, exc_value, exc_tb):
+        try:
+            self.pidfile.close()
+        except IOError as err:
+            if err.errno != 9:
+                raise
+        os.remove(self.path)
+
+# Local variables:
+# coding: utf-8
+# mode: python
+# End:
+# vim: fileencoding=utf-8 filetype=python :

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e4f5cc62/bin/log4j.properties
----------------------------------------------------------------------
diff --git a/bin/log4j.properties b/bin/log4j.properties
index 47d45e8..3f7760a 100644
--- a/bin/log4j.properties
+++ b/bin/log4j.properties
@@ -28,8 +28,8 @@ hadoop.log.dir=.
 # Define the root logger to the system property "plsql.root.logger".
 log4j.rootLogger=${psql.root.logger}
 
-# Logging Threshold
-log4j.threshold=WARN
+# Logging Threshold to INFO for query-server. root logger still at WARN for sqlline clients.
+log4j.threshold=INFO
 
 #
 # Daily Rolling File Appender
@@ -69,3 +69,8 @@ log4j.logger.org.apache.zookeeper=ERROR
 log4j.logger.org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper=ERROR
 log4j.logger.org.apache.hadoop.hbase.zookeeper.ZKUtil=ERROR
 log4j.logger.org.apache.hadoop.hbase.HBaseConfiguration=ERROR
+
+# query server packages
+log4j.logger.org.apache.calcite.avatica=INFO
+log4j.logger.org.apache.phoenix.queryserver.server=INFO
+log4j.logger.org.eclipse.jetty.server=INFO

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e4f5cc62/bin/phoenix_utils.py
----------------------------------------------------------------------
diff --git a/bin/phoenix_utils.py b/bin/phoenix_utils.py
index 5a9145d..055bea6 100755
--- a/bin/phoenix_utils.py
+++ b/bin/phoenix_utils.py
@@ -63,6 +63,10 @@ def setPath():
  hbase_conf_path = os.getenv('HBASE_CONF_PATH','.')
  global testjar
  testjar = find(PHOENIX_TESTS_JAR_PATTERN, phoenix_test_jar_path)
+ global phoenix_queryserver_jar
+ phoenix_queryserver_jar = find("phoenix-server-*-runnable.jar", os.path.join(current_dir, "..", "phoenix-server", "target", "*"))
+ global phoenix_thin_client_jar
+ phoenix_thin_client_jar = find("phoenix-*-query-server-thin-client.jar", os.path.join(current_dir, "..", "phoenix-server-client", "target", "*"))
 
  if phoenix_client_jar == "":
      phoenix_client_jar = findFileInPathWithoutRecursion(PHOENIX_CLIENT_JAR_PATTERN, os.path.join(current_dir, ".."))

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e4f5cc62/bin/queryserver.py
----------------------------------------------------------------------
diff --git a/bin/queryserver.py b/bin/queryserver.py
new file mode 100755
index 0000000..436428b
--- /dev/null
+++ b/bin/queryserver.py
@@ -0,0 +1,147 @@
+#!/usr/bin/env python
+############################################################################
+#
+# 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.
+#
+############################################################################
+
+#
+# Script to handle daemonizing the query server process.
+#
+# usage: queryserver.py [start|stop] [-Dhadoop=configs]
+#
+
+import datetime
+import getpass
+import os
+import os.path
+import signal
+import subprocess
+import sys
+import tempfile
+
+import daemon
+import phoenix_utils
+
+phoenix_utils.setPath()
+
+command = None
+args = sys.argv
+
+if len(args) > 1:
+    if args[1] == 'start':
+        command = 'start'
+    elif args[1] == 'stop':
+        command = 'stop'
+if command:
+    args = args[2:]
+
+if os.name == 'nt':
+    args = subprocess.list2cmdline(args[1:])
+else:
+    import pipes    # pipes module isn't available on Windows
+    args = " ".join([pipes.quote(v) for v in args[1:]])
+
+# HBase configuration folder path (where hbase-site.xml reside) for
+# HBase/Phoenix client side property override
+hbase_config_path = os.getenv('HBASE_CONF_DIR', phoenix_utils.hbase_conf_path)
+
+# default paths ## TODO: add windows support
+hbase_pid_dir = os.path.join(tempfile.gettempdir(), 'phoenix')
+phoenix_log_dir = os.path.join(tempfile.gettempdir(), 'phoenix')
+phoenix_file_basename = 'phoenix-%s-server' % getpass.getuser()
+phoenix_log_file = '%s.log' % phoenix_file_basename
+phoenix_out_file = '%s.out' % phoenix_file_basename
+phoenix_pid_file = '%s.pid' % phoenix_file_basename
+
+# load hbase-env.sh to extract HBASE_PID_DIR
+hbase_env_path = os.path.join(hbase_config_path, 'hbase-env.sh')
+hbase_env = {}
+if os.path.isfile(hbase_env_path):
+    p = subprocess.Popen(['bash', '-c', 'source %s && env' % hbase_env_path], stdout = subprocess.PIPE)
+    for x in p.stdout:
+        (k, v) = x.split('=')
+        hbase_env[k] = v
+
+if hbase_env.has_key('HBASE_PID_DIR'):
+    hbase_pid_dir = hbase_env['HBASE_PID_DIR']
+if hbase_env.has_key('HBASE_LOG_DIR'):
+    phoenix_log_dir = hbase_env['HBASE_LOG_DIR']
+
+log_file_path = os.path.join(phoenix_log_dir, phoenix_log_file)
+out_file_path = os.path.join(phoenix_log_dir, phoenix_out_file)
+pid_file_path = os.path.join(hbase_pid_dir, phoenix_pid_file)
+
+#    " -Xdebug -Xrunjdwp:transport=dt_socket,address=5005,server=y,suspend=n " + \
+#    " -XX:+UnlockCommercialFeatures -XX:+FlightRecorder -XX:FlightRecorderOptions=defaultrecording=true,dumponexit=true" + \
+java_cmd = 'java -cp ' + hbase_config_path + os.pathsep + phoenix_utils.phoenix_queryserver_jar + \
+    " -Dproc_phoenixserver" + \
+    " -Dlog4j.configuration=file:" + os.path.join(phoenix_utils.current_dir, "log4j.properties") + \
+    " -Dpsql.root.logger=%(root_logger)s" + \
+    " -Dpsql.log.dir=%(log_dir)s" + \
+    " -Dpsql.log.file=%(log_file)s" + \
+    " org.apache.phoenix.queryserver.server.Main " + args
+
+if command == 'start':
+    # run in the background
+    d = os.path.dirname(out_file_path)
+    if not os.path.exists(d):
+        os.makedirs(d)
+    with open(out_file_path, 'a+') as out:
+        context = daemon.DaemonContext(
+            pidfile = daemon.PidFile(pid_file_path, 'Query Server already running, PID file found: %s' % pid_file_path),
+            stdout = out,
+            stderr = out,
+        )
+        print 'starting Query Server, logging to %s' % log_file_path
+        with context:
+            # this block is the main() for the forked daemon process
+            child = None
+            cmd = java_cmd % {'root_logger': 'INFO,DRFA', 'log_dir': phoenix_log_dir, 'log_file': phoenix_log_file}
+
+            # notify the child when we're killed
+            def handler(signum, frame):
+                if child:
+                    child.send_signal(signum)
+                sys.exit(0)
+            signal.signal(signal.SIGTERM, handler)
+
+            print '%s launching %s' % (datetime.datetime.now(), cmd)
+            child = subprocess.Popen(cmd.split())
+            sys.exit(child.wait())
+
+elif command == 'stop':
+    if not os.path.isfile(out_file_path):
+        print >> sys.stderr, "no Query Server to stop because PID file not found, %s" % pid_file_path
+        sys.exit(0)
+
+    pid = None
+    with open(pid_file_path, 'r') as p:
+        pid = int(p.read())
+    if not pid:
+        sys.exit("cannot read PID file, %s" % pid_file_path)
+
+    print "stopping Query Server pid %s" % pid
+    with open(out_file_path, 'a+') as out:
+        print >> out, "%s terminating Query Server" % datetime.datetime.now()
+    os.kill(pid, signal.SIGTERM)
+
+else:
+    # run in the foreground using defaults from log4j.properties
+    cmd = java_cmd % {'root_logger': 'INFO,console', 'log_dir': '.', 'log_file': 'psql.log'}
+    child = subprocess.Popen(cmd.split())
+    sys.exit(child.wait())

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e4f5cc62/bin/sqlline-thin.py
----------------------------------------------------------------------
diff --git a/bin/sqlline-thin.py b/bin/sqlline-thin.py
new file mode 100755
index 0000000..2afbb0c
--- /dev/null
+++ b/bin/sqlline-thin.py
@@ -0,0 +1,89 @@
+#!/usr/bin/env python
+############################################################################
+#
+# 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.
+#
+############################################################################
+
+import os
+import subprocess
+import sys
+import phoenix_utils
+import atexit
+import urlparse
+
+global childProc
+childProc = None
+def kill_child():
+    if childProc is not None:
+        childProc.terminate()
+        childProc.kill()
+        if os.name != 'nt':
+            os.system("reset")
+atexit.register(kill_child)
+
+phoenix_utils.setPath()
+
+url = "localhost:8765"
+sqlfile = ""
+
+def usage_and_exit():
+    sys.exit("usage: sqlline-thin.py [host[:port]] [sql_file]")
+
+def cleanup_url(url):
+    parsed = urlparse.urlparse(url)
+    if parsed.scheme == "":
+        url = "http://" + url
+        parsed = urlparse.urlparse(url)
+    if ":" not in parsed.netloc:
+        url = url + ":8765"
+    return url
+
+
+if len(sys.argv) == 1:
+    pass
+elif len(sys.argv) == 2:
+    if os.path.isfile(sys.argv[1]):
+        sqlfile = sys.argv[1]
+    else:
+        url = sys.argv[1]
+elif len(sys.argv) == 3:
+    url = sys.argv[1]
+    sqlfile = sys.argv[2]
+else:
+    usage_and_exit()
+
+url = cleanup_url(url)
+
+if sqlfile != "":
+    sqlfile = "--run=" + sqlfile
+
+colorSetting = "true"
+# disable color setting for windows OS
+if os.name == 'nt':
+    colorSetting = "false"
+
+java_cmd = 'java -cp "' + phoenix_utils.hbase_conf_path + os.pathsep + phoenix_utils.phoenix_thin_client_jar + \
+    '" -Dlog4j.configuration=file:' + \
+    os.path.join(phoenix_utils.current_dir, "log4j.properties") + \
+    " sqlline.SqlLine -d org.apache.phoenix.queryserver.client.Driver " + \
+    " -u jdbc:phoenix:thin:url=" + url + \
+    " -n none -p none --color=" + colorSetting + " --fastConnect=false --verbose=true " + \
+    " --isolation=TRANSACTION_READ_COMMITTED " + sqlfile
+
+exitcode = subprocess.call(java_cmd, shell=True)
+sys.exit(exitcode)

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e4f5cc62/dev/make_rc.sh
----------------------------------------------------------------------
diff --git a/dev/make_rc.sh b/dev/make_rc.sh
index 6755cf8..e23db07 100755
--- a/dev/make_rc.sh
+++ b/dev/make_rc.sh
@@ -74,6 +74,8 @@ rm -rf $(find . -type d -name archive-tmp);
 cp $(find -iname phoenix-$PHOENIX-client-minimal.jar) $DIR_COMMON;
 cp $(find -iname phoenix-$PHOENIX-client-without-hbase.jar) $DIR_COMMON;
 cp $(find -iname phoenix-core-$PHOENIX.jar) $DIR_COMMON;
+cp $(find -iname phoenix-server-$PHOENIX-runnable.jar) $DIR_COMMON;
+cp $(find -iname phoenix-$PHOENIX-query-server-thin-client.jar) $DIR_COMMON;
 
 # Copy release docs
 function_copy() {
@@ -101,6 +103,9 @@ function_copy_hadoop_specific_jars() {
   cp $(find -iname phoenix-$PHOENIX-client.jar) $DIR_HADOOP$1/phoenix-$PHOENIX-client-hadoop$1.jar;
   cp $(find -iname phoenix-$PHOENIX-server.jar) $DIR_HADOOP$1/phoenix-$PHOENIX-server-hadoop$1.jar;
   cp $(find -iname phoenix-core-$PHOENIX-tests.jar) $DIR_HADOOP$1/phoenix-core-$PHOENIX-tests-hadoop$1.jar;
+  cp $(find -iname phoenix-server-$PHOENIX.jar) $DIR_HADOOP$1/phoenix-server-$PHOENIX-hadoop$1.jar;
+  cp $(find -iname phoenix-server-$PHOENIX-tests.jar) $DIR_HADOOP$1/phoenix-server-$PHOENIX-tests-hadoop$1.jar;
+  cp $(find -iname phoenix-server-client-$PHOENIX.jar) $DIR_HADOOP$1/phoenix-server-client-$PHOENIX-hadoop$1.jar;
   cp $(find -iname phoenix-flume-$PHOENIX.jar) $DIR_HADOOP$1/phoenix-flume-$PHOENIX-hadoop$1.jar;
   cp $(find -iname phoenix-flume-$PHOENIX-tests.jar) $DIR_HADOOP$1/phoenix-flume-$PHOENIX-tests-hadoop$1.jar;
   cp $(find -iname phoenix-pig-$PHOENIX.jar) $DIR_HADOOP$1/phoenix-pig-$PHOENIX-hadoop$1.jar;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e4f5cc62/phoenix-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-assembly/pom.xml b/phoenix-assembly/pom.xml
index b3a992e..8d9a965 100644
--- a/phoenix-assembly/pom.xml
+++ b/phoenix-assembly/pom.xml
@@ -150,5 +150,13 @@
       <groupId>org.apache.phoenix</groupId>
       <artifactId>phoenix-spark</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.apache.phoenix</groupId>
+      <artifactId>phoenix-server</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.phoenix</groupId>
+      <artifactId>phoenix-server-client</artifactId>
+    </dependency>
   </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e4f5cc62/phoenix-assembly/src/build/components-minimal.xml
----------------------------------------------------------------------
diff --git a/phoenix-assembly/src/build/components-minimal.xml b/phoenix-assembly/src/build/components-minimal.xml
index 7414f5e..a3d1304 100644
--- a/phoenix-assembly/src/build/components-minimal.xml
+++ b/phoenix-assembly/src/build/components-minimal.xml
@@ -45,6 +45,8 @@
       <excludes>
         <exclude>org.apache.phoenix:phoenix-flume</exclude>
         <exclude>org.apache.phoenix:phoenix-pig</exclude>
+        <exclude>org.apache.phoenix:phoenix-server</exclude>
+        <exclude>org.apache.phoenix:phoenix-server-client</exclude>
       </excludes>
     </dependencySet>
   </dependencySets>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e4f5cc62/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
index 1277151..260e129 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
@@ -662,7 +662,10 @@ public class PhoenixConnection implements Connection, org.apache.phoenix.jdbc.Jd
 
     @Override
     public void setCatalog(String catalog) throws SQLException {
-        throw new SQLFeatureNotSupportedException();
+        if (!this.getCatalog().equalsIgnoreCase(catalog)) {
+            // allow noop calls to pass through.
+            throw new SQLFeatureNotSupportedException();
+        }
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e4f5cc62/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriver.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriver.java
index ff25fae..9e95667 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriver.java
@@ -105,6 +105,11 @@ public abstract class PhoenixEmbeddedDriver implements Driver, org.apache.phoeni
                 if (url.length() == protoLength) {
                     return true;
                 }
+                // Explicitly ignore connections of "jdbc:phoenix:thin"; leave them for
+                // the thin client
+                if (url.startsWith(PhoenixRuntime.JDBC_THIN_PROTOCOL)) {
+                    return false;
+                }
                 // A connection string of the form "jdbc:phoenix://" means that
                 // the driver is remote which isn't supported, so return false.
                 if (!url.startsWith(DNC_JDBC_PROTOCOL_SUFFIX, protoLength)) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e4f5cc62/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 e5ead10..b2736aa 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
@@ -105,6 +105,11 @@ public class PhoenixRuntime {
      * Root for the JDBC URL that the Phoenix accepts accepts.
      */
     public final static String JDBC_PROTOCOL = "jdbc:phoenix";
+    /**
+     * Root for the JDBC URL used by the thin driver. Duplicated here to avoid dependencies
+     * between modules.
+     */
+    public final static String JDBC_THIN_PROTOCOL = "jdbc:phoenix:thin";
     public final static char JDBC_PROTOCOL_TERMINATOR = ';';
     public final static char JDBC_PROTOCOL_SEPARATOR = ':';
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e4f5cc62/phoenix-server-client/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-server-client/pom.xml b/phoenix-server-client/pom.xml
new file mode 100644
index 0000000..60a4c40
--- /dev/null
+++ b/phoenix-server-client/pom.xml
@@ -0,0 +1,62 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.phoenix</groupId>
+    <artifactId>phoenix</artifactId>
+    <version>4.4.0-SNAPSHOT</version>
+  </parent>
+  <artifactId>phoenix-server-client</artifactId>
+  <name>Phoenix Query Server Client</name>
+  <description>A thin JDBC client for interacting with the query server</description>
+
+  <licenses>
+    <license>
+      <name>The Apache Software License, Version 2.0</name>
+      <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
+      <distribution>repo</distribution>
+      <comments />
+    </license>
+  </licenses>
+
+  <organization>
+    <name>Apache Software Foundation</name>
+    <url>http://www.apache.org</url>
+  </organization>
+
+  <build>
+    <plugins>
+      <plugin>
+        <artifactId>maven-assembly-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>thin-client</id>
+            <phase>package</phase>
+            <goals>
+              <goal>single</goal>
+            </goals>
+            <configuration>
+              <attach>false</attach>
+              <finalName>phoenix-${project.version}</finalName>
+              <descriptors>
+                <descriptor>src/build/query-server-thin-client.xml</descriptor>
+              </descriptors>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.calcite</groupId>
+      <artifactId>calcite-avatica</artifactId>
+    </dependency>
+    <dependency>
+      <!-- TODO: don't really want to depend on sqlline, but our bin scripts are primitive -->
+      <groupId>sqlline</groupId>
+      <artifactId>sqlline</artifactId>
+    </dependency>
+  </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e4f5cc62/phoenix-server-client/src/build/query-server-thin-client.xml
----------------------------------------------------------------------
diff --git a/phoenix-server-client/src/build/query-server-thin-client.xml b/phoenix-server-client/src/build/query-server-thin-client.xml
new file mode 100644
index 0000000..f456904
--- /dev/null
+++ b/phoenix-server-client/src/build/query-server-thin-client.xml
@@ -0,0 +1,40 @@
+<?xml version='1.0'?>
+<!--
+
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you under the Apache License, Version 2.0 (the
+ "License"); you may not use this file except in compliance
+ with the License.  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing,
+ software distributed under the License is distributed on an
+ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ KIND, either express or implied.  See the License for the
+ specific language governing permissions and limitations
+ under the License.
+
+-->
+
+<assembly xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.0"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.0 http://maven.apache.org/xsd/assembly-1.1.0.xsd">
+  <id>query-server-thin-client</id>
+  <!-- All the dependencies (unpacked) necessary to run phoenix from a single, stand-alone jar -->
+  <formats>
+    <format>jar</format>
+  </formats>
+  <includeBaseDirectory>false</includeBaseDirectory>
+  <dependencySets>
+    <dependencySet>
+      <outputDirectory>/</outputDirectory>
+      <useProjectArtifact>true</useProjectArtifact>
+      <unpack>true</unpack>
+      <scope>runtime</scope>
+    </dependencySet>
+  </dependencySets>
+</assembly>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e4f5cc62/phoenix-server-client/src/main/java/org/apache/phoenix/queryserver/client/Driver.java
----------------------------------------------------------------------
diff --git a/phoenix-server-client/src/main/java/org/apache/phoenix/queryserver/client/Driver.java b/phoenix-server-client/src/main/java/org/apache/phoenix/queryserver/client/Driver.java
new file mode 100644
index 0000000..5c8f119
--- /dev/null
+++ b/phoenix-server-client/src/main/java/org/apache/phoenix/queryserver/client/Driver.java
@@ -0,0 +1,49 @@
+/*
+ * 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.queryserver.client;
+
+import org.apache.calcite.avatica.DriverVersion;
+
+public class Driver extends org.apache.calcite.avatica.remote.Driver {
+
+  public static final String CONNECT_STRING_PREFIX = "jdbc:phoenix:thin:";
+
+  static {
+    new Driver().register();
+  }
+
+  public Driver() {
+    super();
+  }
+
+  @Override
+  protected DriverVersion createDriverVersion() {
+    return DriverVersion.load(
+        Driver.class,
+        "org-apache-phoenix-remote-jdbc.properties",
+        "Phoenix Remote JDBC Driver",
+        "unknown version",
+        "Apache Phoenix",
+        "unknown version");
+  }
+
+  @Override
+  protected String getConnectStringPrefix() {
+    return CONNECT_STRING_PREFIX;
+  }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e4f5cc62/phoenix-server-client/src/main/java/org/apache/phoenix/queryserver/client/ThinClientUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-server-client/src/main/java/org/apache/phoenix/queryserver/client/ThinClientUtil.java b/phoenix-server-client/src/main/java/org/apache/phoenix/queryserver/client/ThinClientUtil.java
new file mode 100644
index 0000000..0ef1c8c
--- /dev/null
+++ b/phoenix-server-client/src/main/java/org/apache/phoenix/queryserver/client/ThinClientUtil.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.queryserver.client;
+
+/**
+ * Utilities for thin clients.
+ */
+public final class ThinClientUtil {
+
+  private ThinClientUtil() {}
+
+  public static String getConnectionUrl(String hostname, int port) {
+    return getConnectionUrl("http", hostname, port);
+  }
+
+  public static String getConnectionUrl(String protocol, String hostname, int port) {
+    String urlFmt = Driver.CONNECT_STRING_PREFIX + "url=%s://%s:%s";
+    return String.format(urlFmt, protocol, hostname, port);
+  }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e4f5cc62/phoenix-server-client/src/main/resources/META-INF/services/java.sql.Driver
----------------------------------------------------------------------
diff --git a/phoenix-server-client/src/main/resources/META-INF/services/java.sql.Driver b/phoenix-server-client/src/main/resources/META-INF/services/java.sql.Driver
new file mode 100644
index 0000000..f94d657
--- /dev/null
+++ b/phoenix-server-client/src/main/resources/META-INF/services/java.sql.Driver
@@ -0,0 +1 @@
+org.apache.phoenix.queryserver.client.Driver

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e4f5cc62/phoenix-server-client/src/main/resources/version/org-apache-phoenix-remote-jdbc.properties
----------------------------------------------------------------------
diff --git a/phoenix-server-client/src/main/resources/version/org-apache-phoenix-remote-jdbc.properties b/phoenix-server-client/src/main/resources/version/org-apache-phoenix-remote-jdbc.properties
new file mode 100644
index 0000000..7e8eb7e
--- /dev/null
+++ b/phoenix-server-client/src/main/resources/version/org-apache-phoenix-remote-jdbc.properties
@@ -0,0 +1,25 @@
+# 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.
+#
+driver.name=Apache Phoenix Remote JDBC Driver
+driver.version=${pom.version}
+product.name=Apache Phoenix
+product.version=${pom.version}
+jdbc.compliant=true
+driver.version.major=${version.major}
+driver.version.minor=${version.minor}
+database.version.major=${version.major}
+database.version.minor=${version.minor}
+build.timestamp=${build.timestamp}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e4f5cc62/phoenix-server/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-server/pom.xml b/phoenix-server/pom.xml
new file mode 100644
index 0000000..9f6289f
--- /dev/null
+++ b/phoenix-server/pom.xml
@@ -0,0 +1,107 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.phoenix</groupId>
+    <artifactId>phoenix</artifactId>
+    <version>4.4.0-SNAPSHOT</version>
+  </parent>
+  <artifactId>phoenix-server</artifactId>
+  <name>Phoenix Query Server</name>
+  <description>A query server for exposing Phoenix to thin clients</description>
+
+  <licenses>
+    <license>
+      <name>The Apache Software License, Version 2.0</name>
+      <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
+      <distribution>repo</distribution>
+      <comments />
+    </license>
+  </licenses>
+
+  <organization>
+    <name>Apache Software Foundation</name>
+    <url>http://www.apache.org</url>
+  </organization>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>build-helper-maven-plugin</artifactId>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-failsafe-plugin</artifactId>
+      </plugin>
+      <plugin>
+        <artifactId>maven-assembly-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>runnable</id>
+            <phase>package</phase>
+            <goals>
+              <goal>single</goal>
+            </goals>
+            <configuration>
+              <attach>false</attach>
+              <finalName>${project.artifactId}-${project.version}</finalName>
+              <descriptors>
+                <descriptor>src/build/query-server-runnable.xml</descriptor>
+              </descriptors>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.phoenix</groupId>
+      <artifactId>phoenix-server-client</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.calcite</groupId>
+      <artifactId>calcite-avatica</artifactId>
+      <version>${calcite.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.calcite</groupId>
+      <artifactId>calcite-avatica-server</artifactId>
+      <version>${calcite.version}</version>
+    </dependency>
+    <dependency>
+      <!-- TODO: really don't need all of Phoenix, just the thick client. -->
+      <groupId>org.apache.phoenix</groupId>
+      <artifactId>phoenix-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>commons-logging</groupId>
+      <artifactId>commons-logging</artifactId>
+    </dependency>
+    <!-- for tests -->
+    <dependency>
+      <groupId>org.apache.phoenix</groupId>
+      <artifactId>phoenix-core</artifactId>
+      <classifier>tests</classifier>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-it</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-minicluster</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-testing-util</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e4f5cc62/phoenix-server/src/build/query-server-runnable.xml
----------------------------------------------------------------------
diff --git a/phoenix-server/src/build/query-server-runnable.xml b/phoenix-server/src/build/query-server-runnable.xml
new file mode 100644
index 0000000..e2a3dc4
--- /dev/null
+++ b/phoenix-server/src/build/query-server-runnable.xml
@@ -0,0 +1,39 @@
+<?xml version='1.0'?>
+<!--
+
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you under the Apache License, Version 2.0 (the
+ "License"); you may not use this file except in compliance
+ with the License.  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing,
+ software distributed under the License is distributed on an
+ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ KIND, either express or implied.  See the License for the
+ specific language governing permissions and limitations
+ under the License.
+
+-->
+
+<assembly xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.0"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.0 http://maven.apache.org/xsd/assembly-1.1.0.xsd">
+  <id>runnable</id>
+  <formats>
+    <format>jar</format>
+  </formats>
+  <includeBaseDirectory>false</includeBaseDirectory>
+  <dependencySets>
+    <dependencySet>
+      <outputDirectory>/</outputDirectory>
+      <useProjectArtifact>true</useProjectArtifact>
+      <unpack>true</unpack>
+      <scope>runtime</scope>
+    </dependencySet>
+  </dependencySets>
+</assembly>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e4f5cc62/phoenix-server/src/it/java/org/apache/phoenix/end2end/QueryServerBasicsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-server/src/it/java/org/apache/phoenix/end2end/QueryServerBasicsIT.java b/phoenix-server/src/it/java/org/apache/phoenix/end2end/QueryServerBasicsIT.java
new file mode 100644
index 0000000..c3b6115
--- /dev/null
+++ b/phoenix-server/src/it/java/org/apache/phoenix/end2end/QueryServerBasicsIT.java
@@ -0,0 +1,150 @@
+/*
+ * 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.end2end;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.phoenix.queryserver.client.ThinClientUtil;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.Statement;
+import java.util.concurrent.TimeUnit;
+
+import static java.lang.String.format;
+
+import static org.apache.phoenix.query.QueryConstants.SYSTEM_SCHEMA_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_CAT;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_CATALOG;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SCHEM;
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.*;
+
+/**
+ * Smoke test for query server.
+ */
+public class QueryServerBasicsIT extends BaseHBaseManagedTimeIT {
+
+  private static final Log LOG = LogFactory.getLog(QueryServerBasicsIT.class);
+
+  private static QueryServerThread AVATICA_SERVER;
+  private static String CONN_STRING;
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    String url = getUrl();
+    AVATICA_SERVER = new QueryServerThread(new String[] { url }, getTestClusterConfig(),
+        QueryServerBasicsIT.class.getName());
+    AVATICA_SERVER.start();
+    final int port = AVATICA_SERVER.getMain().getPort();
+    AVATICA_SERVER.getMain().awaitRunning();
+    LOG.info("Avatica server started on port " + port);
+    CONN_STRING = ThinClientUtil.getConnectionUrl("localhost", port);
+    LOG.info("JDBC connection string is " + CONN_STRING);
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    if (AVATICA_SERVER != null) {
+      AVATICA_SERVER.join(TimeUnit.MINUTES.toMillis(1));
+      Throwable t = AVATICA_SERVER.getMain().getThrowable();
+      if (t != null) {
+        fail("query server threw. " + t.getMessage());
+      }
+      assertEquals("query server didn't exit cleanly", 0, AVATICA_SERVER.getMain().getRetCode());
+    }
+  }
+
+  @Test
+  public void testCatalogs() throws Exception {
+    try (final Connection connection = DriverManager.getConnection(CONN_STRING)) {
+      assertThat(connection.isClosed(), is(false));
+      try (final ResultSet resultSet = connection.getMetaData().getCatalogs()) {
+        final ResultSetMetaData metaData = resultSet.getMetaData();
+        assertFalse("unexpected populated resultSet", resultSet.next());
+        assertEquals(1, metaData.getColumnCount());
+        assertEquals(TABLE_CAT, metaData.getColumnName(1));
+      }
+    }
+  }
+
+  @Test
+  public void testSchemas() throws Exception {
+    try (final Connection connection = DriverManager.getConnection(CONN_STRING)) {
+      assertThat(connection.isClosed(), is(false));
+      try (final ResultSet resultSet = connection.getMetaData().getSchemas()) {
+        final ResultSetMetaData metaData = resultSet.getMetaData();
+        assertTrue("unexpected empty resultset", resultSet.next());
+        assertEquals(2, metaData.getColumnCount());
+        assertEquals(TABLE_SCHEM, metaData.getColumnName(1));
+        assertEquals(TABLE_CATALOG, metaData.getColumnName(2));
+        boolean containsSystem = false;
+        do {
+          if (resultSet.getString(1).equalsIgnoreCase(SYSTEM_SCHEMA_NAME)) containsSystem = true;
+        } while (resultSet.next());
+        assertTrue(format("should contain at least %s schema.", SYSTEM_SCHEMA_NAME), containsSystem);
+      }
+    }
+  }
+
+  @Test
+  public void smokeTest() throws Exception {
+    try (final Connection connection = DriverManager.getConnection(CONN_STRING)) {
+      assertThat(connection.isClosed(), is(false));
+      connection.setAutoCommit(true);
+      try (final Statement stmt = connection.createStatement()) {
+        assertFalse(stmt.execute("CREATE TABLE TEST_TABLE("
+            + "id INTEGER NOT NULL, "
+            + "pk varchar(3) NOT NULL "
+            + "CONSTRAINT PK_CONSTRAINT PRIMARY KEY (id, pk))"));
+        assertEquals(0, stmt.getUpdateCount());
+        assertEquals(1, stmt.executeUpdate("UPSERT INTO TEST_TABLE VALUES(1, 'foo')"));
+        assertEquals(1, stmt.executeUpdate("UPSERT INTO TEST_TABLE VALUES(2, 'bar')"));
+        assertTrue(stmt.execute("SELECT * FROM TEST_TABLE"));
+        try (final ResultSet resultSet = stmt.getResultSet()) {
+          assertTrue(resultSet.next());
+          assertEquals(1, resultSet.getInt(1));
+          assertEquals("foo", resultSet.getString(2));
+          assertTrue(resultSet.next());
+          assertEquals(2, resultSet.getInt(1));
+          assertEquals("bar", resultSet.getString(2));
+        }
+      }
+      final String sql = "SELECT * FROM TEST_TABLE WHERE id = ?";
+      try (final PreparedStatement stmt = connection.prepareStatement(sql)) {
+        stmt.setInt(1, 1);
+        try (ResultSet resultSet = stmt.executeQuery()) {
+          assertTrue(resultSet.next());
+          assertEquals(1, resultSet.getInt(1));
+          assertEquals("foo", resultSet.getString(2));
+        }
+        stmt.clearParameters();
+        stmt.setInt(1, 5);
+        try (final ResultSet resultSet = stmt.executeQuery()) {
+          assertFalse(resultSet.next());
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e4f5cc62/phoenix-server/src/it/java/org/apache/phoenix/end2end/QueryServerThread.java
----------------------------------------------------------------------
diff --git a/phoenix-server/src/it/java/org/apache/phoenix/end2end/QueryServerThread.java b/phoenix-server/src/it/java/org/apache/phoenix/end2end/QueryServerThread.java
new file mode 100644
index 0000000..b55af90
--- /dev/null
+++ b/phoenix-server/src/it/java/org/apache/phoenix/end2end/QueryServerThread.java
@@ -0,0 +1,51 @@
+/*
+ * 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.end2end;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.phoenix.queryserver.server.Main;
+
+import java.util.concurrent.TimeUnit;
+
+/** Wraps up the query server for tests. */
+public class QueryServerThread extends Thread {
+
+  private static final Log LOG = LogFactory.getLog(QueryServerThread.class);
+
+  private final Main main;
+
+  public QueryServerThread(String[] argv, Configuration conf) {
+    this(argv, conf, null);
+  }
+
+  public QueryServerThread(String[] argv, Configuration conf, String name) {
+    this(new Main(argv, conf), name);
+  }
+
+  private QueryServerThread(Main m, String name) {
+    super(m, "query server" + (name == null ? "" : (" - " + name)));
+    this.main = m;
+    setDaemon(true);
+  }
+
+  public Main getMain() {
+    return main;
+  }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e4f5cc62/phoenix-server/src/main/java/org/apache/phoenix/queryserver/server/Main.java
----------------------------------------------------------------------
diff --git a/phoenix-server/src/main/java/org/apache/phoenix/queryserver/server/Main.java b/phoenix-server/src/main/java/org/apache/phoenix/queryserver/server/Main.java
new file mode 100644
index 0000000..52529d5
--- /dev/null
+++ b/phoenix-server/src/main/java/org/apache/phoenix/queryserver/server/Main.java
@@ -0,0 +1,158 @@
+/*
+ * 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.queryserver.server;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.calcite.avatica.Meta;
+import org.apache.calcite.avatica.remote.LocalService;
+import org.apache.calcite.avatica.remote.Service;
+import org.apache.calcite.avatica.server.AvaticaHandler;
+import org.apache.calcite.avatica.server.HttpServer;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.util.Strings;
+import org.apache.hadoop.net.DNS;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+import java.util.Arrays;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * A query server for Phoenix over Calcite's Avatica.
+ */
+public final class Main extends Configured implements Tool, Runnable {
+
+  public static final String QUERY_SERVER_META_FACTORY_KEY =
+      "phoenix.queryserver.metafactory.class";
+
+  public static final String QUERY_SERVER_HTTP_PORT_KEY =
+      "phoenix.queryserver.http.port";
+  public static final int DEFAULT_HTTP_PORT = 8765;
+
+  public static final String KEYTAB_FILENAME_KEY = "phoenix.queryserver.keytab.file";
+  public static final String KERBEROS_PRINCIPAL_KEY = "phoenix.queryserver.kerberos.principal";
+  public static final String DNS_NAMESERVER_KEY = "phoenix.queryserver.dns.nameserver";
+  public static final String DNS_INTERFACE_KEY = "phoenix.queryserver.dns.interface";
+  public static final String HBASE_SECURITY_CONF_KEY = "hbase.security.authentication";
+
+  protected static final Log LOG = LogFactory.getLog(Main.class);
+
+  private final String[] argv;
+  private final CountDownLatch runningLatch = new CountDownLatch(1);
+  private int port = DEFAULT_HTTP_PORT;
+  private int retCode = 0;
+  private Throwable t = null;
+
+  /** Constructor for use from {@link org.apache.hadoop.util.ToolRunner}. */
+  public Main() {
+    this(null, null);
+  }
+
+  /** Constructor for use as {@link java.lang.Runnable}. */
+  public Main(String[] argv, Configuration conf) {
+    this.argv = argv;
+    setConf(conf);
+  }
+
+  /**
+   * @return the port number this instance is bound to.
+   */
+  @VisibleForTesting
+  public int getPort() {
+    return port;
+  }
+
+  /**
+   * @return the return code from running as a {@link Tool}.
+   */
+  @VisibleForTesting
+  public int getRetCode() {
+    return retCode;
+  }
+
+  /**
+   * @return the throwable from an unsuccessful run, or null otherwise.
+   */
+  @VisibleForTesting
+  public Throwable getThrowable() {
+    return t;
+  }
+
+  /** Calling thread waits until the server is running. */
+  public void awaitRunning() throws InterruptedException {
+    runningLatch.await();
+  }
+
+  /** Calling thread waits until the server is running. */
+  public void awaitRunning(long timeout, TimeUnit unit) throws InterruptedException {
+    runningLatch.await(timeout, unit);
+  }
+
+  @Override
+  public int run(String[] args) throws Exception {
+    try {
+      // handle secure cluster credentials
+      if ("kerberos".equalsIgnoreCase(getConf().get(HBASE_SECURITY_CONF_KEY))) {
+        String hostname = Strings.domainNamePointerToHostName(DNS.getDefaultHost(
+            getConf().get(DNS_INTERFACE_KEY, "default"),
+            getConf().get(DNS_NAMESERVER_KEY, "default")));
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Login to " + hostname + " using " + getConf().get(KEYTAB_FILENAME_KEY)
+              + " and principal " + getConf().get(KERBEROS_PRINCIPAL_KEY) + ".");
+          SecurityUtil.login(getConf(), KEYTAB_FILENAME_KEY, KERBEROS_PRINCIPAL_KEY, hostname);
+        }
+        LOG.info("Login successful.");
+      }
+      Class<? extends PhoenixMetaFactory> factoryClass = getConf().getClass(
+          QUERY_SERVER_META_FACTORY_KEY, PhoenixMetaFactoryImpl.class, PhoenixMetaFactory.class);
+      port = getConf().getInt(QUERY_SERVER_HTTP_PORT_KEY, DEFAULT_HTTP_PORT);
+      PhoenixMetaFactory factory =
+          factoryClass.getDeclaredConstructor(Configuration.class).newInstance(getConf());
+      Meta meta = factory.create(Arrays.asList(args));
+      Service service = new LocalService(meta);
+      HttpServer server = new HttpServer(port, new AvaticaHandler(service));
+      server.start();
+      runningLatch.countDown();
+      server.join();
+      return 0;
+    } catch (Throwable t) {
+      LOG.fatal("Unrecoverable service error. Shutting down.", t);
+      this.t = t;
+      return -1;
+    }
+  }
+
+  @Override public void run() {
+    try {
+      retCode = run(argv);
+    } catch (Exception e) {
+      // already logged
+    }
+  }
+
+  public static void main(String[] argv) throws Exception {
+    int ret = ToolRunner.run(HBaseConfiguration.create(), new Main(), argv);
+    System.exit(ret);
+  }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e4f5cc62/phoenix-server/src/main/java/org/apache/phoenix/queryserver/server/PhoenixMetaFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-server/src/main/java/org/apache/phoenix/queryserver/server/PhoenixMetaFactory.java b/phoenix-server/src/main/java/org/apache/phoenix/queryserver/server/PhoenixMetaFactory.java
new file mode 100644
index 0000000..02344a3
--- /dev/null
+++ b/phoenix-server/src/main/java/org/apache/phoenix/queryserver/server/PhoenixMetaFactory.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.queryserver.server;
+
+import org.apache.calcite.avatica.Meta;
+import org.apache.hadoop.conf.Configurable;
+
+/**
+ * A @{link Meta.Factory} that can also respect Hadoop
+ * {@link org.apache.hadoop.conf.Configuration} objects.
+ */
+public interface PhoenixMetaFactory extends Meta.Factory, Configurable {
+}