You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by pa...@apache.org on 2016/11/01 20:29:46 UTC

[01/15] drill git commit: DRILL-3423: Initial HTTPD log plugin. Needs tests. Would be good to improve the timestamp and cookies behaviors since we can make those more type specific.

Repository: drill
Updated Branches:
  refs/heads/master 1e6fa00cd -> 83513daf0


DRILL-3423: Initial HTTPD log plugin. Needs tests. Would be good to improve the timestamp and cookies behaviors since we can make those more type specific.


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

Branch: refs/heads/master
Commit: 818f94507b9d1792282a4bb8abd2d22d2fc2eb57
Parents: 1e6fa00
Author: Jacques Nadeau <ja...@apache.org>
Authored: Tue Jul 28 15:29:32 2015 -0700
Committer: Parth Chandra <pa...@apache.org>
Committed: Tue Nov 1 10:42:21 2016 -0700

----------------------------------------------------------------------
 exec/java-exec/pom.xml                          |   6 +
 .../exec/store/dfs/easy/EasyFormatPlugin.java   |  15 +-
 .../exec/store/httpd/HttpdFormatPlugin.java     | 487 +++++++++++++++++++
 .../resources/bootstrap-storage-plugins.json    |   4 +
 .../drill/exec/store/httpd/TestHttpdPlugin.java |  31 ++
 .../resources/bootstrap-storage-plugins.json    |   4 +
 .../test/resources/store/httpd/example1.httpd   |   1 +
 7 files changed, 542 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/818f9450/exec/java-exec/pom.xml
----------------------------------------------------------------------
diff --git a/exec/java-exec/pom.xml b/exec/java-exec/pom.xml
index ceb0b62..e9b07be 100644
--- a/exec/java-exec/pom.xml
+++ b/exec/java-exec/pom.xml
@@ -453,6 +453,12 @@
         </exclusion>
       </exclusions>
     </dependency>
+    <dependency>
+      <groupId>nl.basjes.parse.httpdlog</groupId>
+      <artifactId>httpdlog-parser</artifactId>
+      <version>2.1.1</version>
+    </dependency>
+        
   </dependencies>
 
   <profiles>

http://git-wip-us.apache.org/repos/asf/drill/blob/818f9450/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
index f95a323..c7f9cf8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
@@ -52,11 +52,14 @@ import org.apache.drill.exec.store.dfs.FormatMatcher;
 import org.apache.drill.exec.store.dfs.FormatPlugin;
 import org.apache.drill.exec.store.schedule.CompleteFileWork;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
 
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
 
 public abstract class EasyFormatPlugin<T extends FormatPluginConfig> implements FormatPlugin {
+
+  @SuppressWarnings("unused")
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(EasyFormatPlugin.class);
 
   private final BasicFormatMatcher matcher;
@@ -66,7 +69,7 @@ public abstract class EasyFormatPlugin<T extends FormatPluginConfig> implements
   private final boolean blockSplittable;
   private final Configuration fsConf;
   private final StoragePluginConfig storageConfig;
-  protected final FormatPluginConfig formatConfig;
+  protected final T formatConfig;
   private final String name;
   private final boolean compressible;
 
@@ -129,7 +132,7 @@ public abstract class EasyFormatPlugin<T extends FormatPluginConfig> implements
       scan = new EasySubScan(scan.getUserName(), scan.getWorkUnits(), scan.getFormatPlugin(),
           columnExplorer.getTableColumns(), scan.getSelectionRoot());
       scan.setOperatorId(scan.getOperatorId());
-    }
+        }
 
     OperatorContext oContext = context.newOperatorContext(scan);
     final DrillFileSystem dfs;
@@ -142,21 +145,21 @@ public abstract class EasyFormatPlugin<T extends FormatPluginConfig> implements
     List<RecordReader> readers = Lists.newArrayList();
     List<Map<String, String>> implicitColumns = Lists.newArrayList();
     Map<String, String> mapWithMaxColumns = Maps.newLinkedHashMap();
-    for(FileWork work : scan.getWorkUnits()) {
+    for(FileWork work : scan.getWorkUnits()){
       RecordReader recordReader = getRecordReader(context, dfs, work, scan.getColumns(), scan.getUserName());
       readers.add(recordReader);
       Map<String, String> implicitValues = columnExplorer.populateImplicitColumns(work, scan.getSelectionRoot());
       implicitColumns.add(implicitValues);
       if (implicitValues.size() > mapWithMaxColumns.size()) {
         mapWithMaxColumns = implicitValues;
+        }
       }
-    }
 
     // all readers should have the same number of implicit columns, add missing ones with value null
     Map<String, String> diff = Maps.transformValues(mapWithMaxColumns, Functions.constant((String) null));
     for (Map<String, String> map : implicitColumns) {
       map.putAll(Maps.difference(map, diff).entriesOnlyOnRight());
-    }
+      }
 
     return new ScanBatch(scan, context, oContext, readers.iterator(), implicitColumns);
   }
@@ -194,7 +197,7 @@ public abstract class EasyFormatPlugin<T extends FormatPluginConfig> implements
   }
 
   @Override
-  public FormatPluginConfig getConfig() {
+  public T getConfig() {
     return formatConfig;
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/818f9450/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdFormatPlugin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdFormatPlugin.java
new file mode 100644
index 0000000..7b8dc0e
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdFormatPlugin.java
@@ -0,0 +1,487 @@
+/**
+ * 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.drill.exec.store.httpd;
+
+import io.netty.buffer.DrillBuf;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+import nl.basjes.parse.core.Parser;
+import nl.basjes.parse.core.exceptions.DissectionFailure;
+import nl.basjes.parse.core.exceptions.InvalidDissectorException;
+import nl.basjes.parse.core.exceptions.MissingDissectorsException;
+import nl.basjes.parse.httpdlog.ApacheHttpdLogFormatDissector;
+import nl.basjes.parse.httpdlog.dissectors.HttpFirstLineDissector;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.logical.FormatPluginConfig;
+import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.physical.impl.OutputMutator;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.store.AbstractRecordReader;
+import org.apache.drill.exec.store.RecordWriter;
+import org.apache.drill.exec.store.dfs.DrillFileSystem;
+import org.apache.drill.exec.store.dfs.easy.EasyFormatPlugin;
+import org.apache.drill.exec.store.dfs.easy.EasyWriter;
+import org.apache.drill.exec.store.dfs.easy.FileWork;
+import org.apache.drill.exec.vector.complex.impl.VectorContainerWriter;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter.ComplexWriter;
+import org.apache.drill.exec.vector.complex.writer.BigIntWriter;
+import org.apache.drill.exec.vector.complex.writer.Float8Writer;
+import org.apache.drill.exec.vector.complex.writer.VarCharWriter;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.LineRecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.TextInputFormat;
+
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import com.google.common.base.Charsets;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+public class HttpdFormatPlugin extends EasyFormatPlugin<HttpdFormatPlugin.HttpdLogFormatConfig> {
+
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HttpdFormatPlugin.class);
+
+  private static final String DEFAULT_EXTENSION = "httpd";
+
+  public HttpdFormatPlugin(
+      String name,
+      DrillbitContext context,
+      Configuration fsConf,
+      StoragePluginConfig storageConfig,
+      HttpdLogFormatConfig formatConfig) {
+    super(name, context, fsConf, storageConfig, formatConfig, true, false, true, true,
+        Lists.newArrayList(DEFAULT_EXTENSION), DEFAULT_EXTENSION);
+  }
+
+  @JsonTypeName("httpd")
+  public static class HttpdLogFormatConfig implements FormatPluginConfig {
+    public String format;
+  }
+
+  private class RecordReader extends AbstractRecordReader {
+
+    private final DrillFileSystem fs;
+    private final FileWork work;
+    private final FragmentContext fragmentContext;
+
+    private ComplexWriter writer;
+    private Parser<ComplexWriterFacade> parser;
+    private LineRecordReader lineReader;
+    private LongWritable lineNumber;
+    private ComplexWriterFacade record;
+    private DrillBuf managedBuffer;
+
+    public RecordReader(FragmentContext context, DrillFileSystem fs, FileWork work) {
+      this.fs = fs;
+      this.work = work;
+      fragmentContext = context;
+      managedBuffer = context.getManagedBuffer();
+    }
+
+    @Override
+    public void setup(OperatorContext context, OutputMutator output) throws ExecutionSetupException {
+
+      try {
+
+        parser = new PartiallyDissectedParser<ComplexWriterFacade>(
+            ComplexWriterFacade.class,
+            HttpdFormatPlugin.this.getConfig().format);
+        writer = new VectorContainerWriter(output);
+        record = new ComplexWriterFacade(writer);
+        record.addAsParseTarget(parser);
+
+        final Path path = fs.makeQualified(new Path(work.getPath()));
+        FileSplit split = new FileSplit(path, work.getStart(), work.getLength(), new String[] { "" });
+        TextInputFormat inputFormat = new TextInputFormat();
+        JobConf job = new JobConf(fs.getConf());
+        job.setInt("io.file.buffer.size", fragmentContext.getConfig()
+            .getInt(ExecConstants.TEXT_LINE_READER_BUFFER_SIZE));
+        job.setInputFormat(inputFormat.getClass());
+        lineReader = (LineRecordReader) inputFormat.getRecordReader(split, job, Reporter.NULL);
+        lineNumber = lineReader.createKey();
+      } catch (Exception e) {
+        throw handleAndGenerate("Failure in creating record reader", e);
+      }
+
+
+    }
+
+    private DrillBuf buf(int size) {
+      if (managedBuffer.capacity() < size) {
+        managedBuffer = managedBuffer.reallocIfNeeded(size);
+      }
+      return managedBuffer;
+    }
+
+    protected RuntimeException handleAndGenerate(String s, Exception e) {
+      throw UserException.dataReadError(e)
+          .message(s + "\n%s", e.getMessage())
+          .addContext("Path", work.getPath())
+          .addContext("Split Start", work.getStart())
+          .addContext("Split Length", work.getLength())
+          .addContext("Local Line Number", lineNumber.get())
+          .build(logger);
+    }
+
+    @Override
+    public int next() {
+      try {
+        final Text currentText = lineReader.createValue();
+
+        writer.allocate();
+        writer.reset();
+        int recordCount = 0;
+
+        for (; recordCount < 4095 && lineReader.next(lineNumber, currentText); recordCount++) {
+          writer.setPosition(recordCount);
+          parser.parse(record, currentText.toString());
+        }
+
+        writer.setValueCount(recordCount);
+        return recordCount;
+      } catch (DissectionFailure | InvalidDissectorException | MissingDissectorsException | IOException e) {
+        throw handleAndGenerate("Failure while reading httpd log record.", e);
+      }
+    }
+
+    @Override
+    public void cleanup() {
+
+      try {
+        if (lineReader != null) {
+          lineReader.close();
+        }
+      } catch (IOException e) {
+        logger.warn("Failure while closing Httpd reader.", e);
+      }
+    }
+
+    /**
+     * Maps Httpd Log Libraries calls to Drills ComplexWriter interface.
+     */
+    public class ComplexWriterFacade {
+      private final ComplexWriter writer;
+      private final Map<String, VarCharWriter> stringWriters = Maps.newHashMap();
+      private final Map<String, BigIntWriter> longWriters = Maps.newHashMap();
+      private final Map<String, Float8Writer> doubleWriters = Maps.newHashMap();
+
+      private ComplexWriterFacade(ComplexWriter writer) {
+        this.writer = writer;
+      }
+
+      @SuppressWarnings("unused")
+      public void set(final String name, final String value) {
+        if (value != null) {
+          final byte[] stringBytes = value.getBytes(Charsets.UTF_8);
+          final DrillBuf stringBuffer = buf(stringBytes.length);
+          stringBuffer.clear();
+          stringBuffer.writeBytes(stringBytes);
+          final VarCharWriter writer = stringWriters.get(name);
+          if (writer != null) {
+            writer.writeVarChar(0, stringBytes.length, stringBuffer);
+          } else {
+            logger.warn("Dropped string.  Name: {}, Value: {}", name, value);
+          }
+        }
+      }
+
+      @SuppressWarnings("unused")
+      public void set(String name, Long value) {
+        if (value != null) {
+          longWriters.get(name).writeBigInt(value);
+        }
+      }
+
+      @SuppressWarnings("unused")
+      public void set(String name, Double value) {
+        if (value != null) {
+          doubleWriters.get(name).writeFloat8(value);
+        }
+      }
+
+      private void add(Parser<ComplexWriterFacade> parser, String path, VarCharWriter writer)
+          throws NoSuchMethodException,
+          SecurityException {
+        stringWriters.put(path, writer);
+        parser.addParseTarget(
+            ComplexWriterFacade.class.getMethod("set", new Class[] { String.class, String.class }),
+            path);
+      }
+
+      @SuppressWarnings("unused")
+      private void add(Parser<ComplexWriterFacade> parser, String path, Float8Writer writer)
+          throws NoSuchMethodException,
+          SecurityException {
+        doubleWriters.put(path, writer);
+        parser.addParseTarget(
+            ComplexWriterFacade.class.getMethod("set", new Class[] { String.class, Double.class }),
+            path);
+      }
+
+      private void add(Parser<ComplexWriterFacade> parser, String path, BigIntWriter writer)
+          throws NoSuchMethodException,
+          SecurityException {
+        longWriters.put(path, writer);
+        parser.addParseTarget(
+            ComplexWriterFacade.class.getMethod("set", new Class[] { String.class, Long.class }),
+            path);
+      }
+
+      public void addAsParseTarget(Parser<ComplexWriterFacade> parser) {
+        try {
+
+          for (final String path : parser.getPossiblePaths()) {
+            switch (path) {
+            case "IP:connection.client.ip":
+              add(parser, path, writer.rootAsMap().map("client").varChar("ip"));
+              break;
+            case "IP:connection.client.peerip":
+              add(parser, path, writer.rootAsMap().map("client").varChar("peer_ip"));
+              break;
+            case "IP:connection.server.ip":
+              add(parser, path, writer.rootAsMap().map("server").varChar("ip"));
+              break;
+            case "BYTES:response.body.bytes":
+              add(parser, path, writer.rootAsMap().map("response").bigInt("bytes"));
+              break;
+            case "BYTES:response.body.bytesclf":
+              add(parser, path, writer.rootAsMap().map("response").bigInt("bytes"));
+              break;
+            case "HTTP.COOKIE:request.cookies.":
+              add(parser, path, writer.rootAsMap().map("request").varChar("cookies"));
+              break;
+            case "MICROSECONDS:server.process.time":
+              add(parser, path, writer.rootAsMap().map("response").bigInt("process_time"));
+              break;
+            case "FILENAME:server.filename":
+              add(parser, path, writer.rootAsMap().map("response").varChar("filename"));
+              break;
+            case "IP:connection.client.host":
+              add(parser, path, writer.rootAsMap().map("client").varChar("host"));
+              break;
+            case "PROTOCOL:request.protocol":
+              add(parser, path, writer.rootAsMap().map("request").varChar("protocol"));
+              break;
+            case "HTTP.HEADER:request.header.":
+              add(parser, path, writer.rootAsMap().map("request").varChar("header"));
+              break;
+            case "NUMBER:connection.keepalivecount":
+              add(parser, path, writer.rootAsMap().map("client").bigInt("keepalivecount"));
+              break;
+            case "NUMBER:connection.client.logname":
+              add(parser, path, writer.rootAsMap().map("request").bigInt("logname"));
+              break;
+            case "STRING:request.errorlogid":
+              add(parser, path, writer.rootAsMap().map("request").varChar("errorlogid"));
+              break;
+            case "HTTP.METHOD:request.method":
+              add(parser, path, writer.rootAsMap().map("request").varChar("method"));
+              break;
+            case "PORT:request.server.port.canonical":
+              add(parser, path, writer.rootAsMap().map("server").bigInt("canonical_port"));
+              break;
+            case "PORT:connection.server.port.canonical":
+              add(parser, path, writer.rootAsMap().map("server").bigInt("canonical_port"));
+              break;
+            case "PORT:connection.client.port":
+              add(parser, path, writer.rootAsMap().map("client").bigInt("port"));
+              break;
+            case "NUBMER:connection.server.child.processid":
+              add(parser, path, writer.rootAsMap().map("server").bigInt("process_id"));
+              break;
+            case "NUMBER:connection.server.child.threadid":
+              add(parser, path, writer.rootAsMap().map("server").bigInt("thread_id"));
+              break;
+            case "STRING:connection.server.child.hexthreadid":
+              add(parser, path, writer.rootAsMap().map("connection").varChar("hex_thread_id"));
+              break;
+            case "HTTP.QUERYSTRING:request.querystring":
+              add(parser, path, writer.rootAsMap().map("").varChar(""));
+              break;
+            case "HTTP.FIRSTLINE:request.firstline":
+              add(parser, path, writer.rootAsMap().map("").varChar(""));
+              break;
+            case "STRING:request.handler":
+              add(parser, path, writer.rootAsMap().map("request").varChar("handler"));
+              break;
+            case "STRING:request.status.original":
+              add(parser, path, writer.rootAsMap().map("request").varChar("status_original"));
+              break;
+            case "STRING:request.status.last":
+              add(parser, path, writer.rootAsMap().map("request").varChar("status_last"));
+              break;
+            case "TIME.STAMP:request.receive.time":
+              add(parser, path, writer.rootAsMap().map("request").varChar("timestamp"));
+              break;
+            case "TIME.EPOCH:request.receive.time.begin.msec":
+              add(parser, path, writer.rootAsMap().map("request").bigInt("begin_msec"));
+              break;
+            case "TIME.EPOCH:request.receive.time.end.msec":
+              add(parser, path, writer.rootAsMap().map("request").bigInt("end_msec"));
+              break;
+            case "TIME.EPOCH.USEC:request.receive.time.begin.usec":
+              add(parser, path, writer.rootAsMap().map("request").bigInt("begin_usec"));
+              break;
+            case "TIME.EPOCH.USEC:request.receive.time.end.usec":
+              add(parser, path, writer.rootAsMap().map("request").bigInt("end_usec"));
+              break;
+            case "TIME.EPOCH:request.receive.time.begin.msec_frac":
+              add(parser, path, writer.rootAsMap().map("request").bigInt("begin_msec_frac"));
+              break;
+            case "TIME.EPOCH:request.receive.time.end.msec_frac":
+              add(parser, path, writer.rootAsMap().map("request").varChar("end_msec_frac"));
+              break;
+            case "TIME.EPOCH.USEC_FRAC:request.receive.time.begin.usec_frac":
+              add(parser, path, writer.rootAsMap().map("request").varChar("begin_usec_frac"));
+              break;
+            case "TIME.EPOCH.USEC_FRAC:request.receive.time.end.usec_frac":
+              add(parser, path, writer.rootAsMap().map("request").varChar("end_usec_frac"));
+              break;
+            case "SECONDS:response.server.processing.time":
+              add(parser, path, writer.rootAsMap().map("response").varChar("processing_time"));
+              break;
+            case "STRING:connection.client.user":
+              add(parser, path, writer.rootAsMap().map("client").varChar("user"));
+              break;
+            case "URI:request.urlpath":
+              add(parser, path, writer.rootAsMap().map("request").varChar("url"));
+              break;
+            case "STRING:connection.server.name.canonical":
+              add(parser, path, writer.rootAsMap().map("server").varChar("canonical_name"));
+              break;
+            case "STRING:connection.server.name":
+              add(parser, path, writer.rootAsMap().map("server").varChar("name"));
+              break;
+            case "HTTP.CONNECTSTATUS:response.connection.status":
+              add(parser, path, writer.rootAsMap().map("response").varChar("connection_status"));
+              break;
+            case "BYTES:request.bytes":
+              add(parser, path, writer.rootAsMap().map("request").varChar("bytes"));
+              break;
+            case "BYTES:response.bytes":
+              add(parser, path, writer.rootAsMap().map("response").bigInt("bytes"));
+              break;
+            case "HTTP.COOKIES:request.cookies":
+              add(parser, path, writer.rootAsMap().map("request").varChar("cookies"));
+              break;
+            case "HTTP.SETCOOKIES:response.cookies":
+              add(parser, path, writer.rootAsMap().map("response").varChar("cookies"));
+              break;
+            case "HTTP.USERAGENT:request.user-agent":
+              add(parser, path, writer.rootAsMap().map("request").varChar("useragent"));
+              break;
+            case "HTTP.URI:request.referer":
+              add(parser, path, writer.rootAsMap().map("request").varChar("referer"));
+              break;
+            case "HTTP.METHOD:method":
+              add(parser, path, writer.rootAsMap().map("request").varChar("method"));
+              break;
+            case "HTTP.URI:uri":
+              add(parser, path, writer.rootAsMap().map("request").varChar("uri"));
+              break;
+            case "HTTP.PROTOCOL:protocol":
+              add(parser, path, writer.rootAsMap().map("request").varChar("protocol"));
+              break;
+            case "HTTP.PROTOCOL.VERSION:protocol.version":
+              add(parser, path, writer.rootAsMap().map("request").varChar("protocol_version"));
+              break;
+            case "HTTP.METHOD:request.firstline.method":
+              add(parser, path, writer.rootAsMap().map("request").varChar("method"));
+              break;
+            case "HTTP.URI:request.firstline.uri":
+              add(parser, path, writer.rootAsMap().map("request").varChar("uri"));
+              break;
+            case "HTTP.PROTOCOL:request.firstline.protocol":
+              add(parser, path, writer.rootAsMap().map("request").varChar("protocol"));
+              break;
+            case "HTTP.PROTOCOL.VERSION:request.firstline.protocol.version":
+              add(parser, path, writer.rootAsMap().map("request").varChar("protocol_version"));
+              break;
+            default:
+
+              // if we don't know what to do, just write the raw value.
+              parser.addParseTarget(
+                  ComplexWriterFacade.class.getMethod("set", new Class[] { String.class, String.class }),
+                  path);
+              final String noPeriodPath = path.replace(".", "_");
+              stringWriters.put(path, writer.rootAsMap().varChar(noPeriodPath));
+              break;
+
+            }
+          }
+
+
+        } catch (MissingDissectorsException | SecurityException | NoSuchMethodException | InvalidDissectorException e) {
+          throw handleAndGenerate("Failure while setting up log mappings.", e);
+        }
+      }
+    }
+  }
+
+  @Override
+  public boolean supportsPushDown() {
+    return true;
+  }
+
+
+  @Override
+  public RecordReader getRecordReader(FragmentContext context, DrillFileSystem dfs,
+      FileWork fileWork, List<SchemaPath> columns) throws ExecutionSetupException {
+    return new RecordReader(context, dfs, fileWork);
+  }
+
+  @Override
+  public RecordWriter getRecordWriter(FragmentContext context, EasyWriter writer) throws IOException {
+    throw new UnsupportedOperationException("Drill doesn't currently support writing to HTTPD logs.");
+  }
+
+  @Override
+  public int getReaderOperatorType() {
+    return -1;
+  }
+
+  @Override
+  public int getWriterOperatorType() {
+    return -1;
+  }
+
+  private class PartiallyDissectedParser<RECORD> extends Parser<RECORD> {
+    public PartiallyDissectedParser(Class<RECORD> clazz, final String logformat) {
+      super(clazz);
+
+      addDissector(new ApacheHttpdLogFormatDissector(logformat));
+      addDissector(new HttpFirstLineDissector());
+      setRootType(ApacheHttpdLogFormatDissector.INPUT_TYPE);
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/818f9450/exec/java-exec/src/main/resources/bootstrap-storage-plugins.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/bootstrap-storage-plugins.json b/exec/java-exec/src/main/resources/bootstrap-storage-plugins.json
index 13d29ea..dab4ada 100644
--- a/exec/java-exec/src/main/resources/bootstrap-storage-plugins.json
+++ b/exec/java-exec/src/main/resources/bootstrap-storage-plugins.json
@@ -29,6 +29,10 @@
           extensions: [ "tsv" ],
           delimiter: "\t"
         },
+        "httpd" :{
+          type: "httpd",
+          format: "%h %l %u %t \"%r\" %>s %b \"%{Referer}i\" \"%{User-Agent}i\" \"%{Cookie}i\""
+        },
         "parquet" : {
           type: "parquet"
         },

http://git-wip-us.apache.org/repos/asf/drill/blob/818f9450/exec/java-exec/src/test/java/org/apache/drill/exec/store/httpd/TestHttpdPlugin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/httpd/TestHttpdPlugin.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/httpd/TestHttpdPlugin.java
new file mode 100644
index 0000000..ce1f685
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/httpd/TestHttpdPlugin.java
@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.store.httpd;
+
+import org.apache.drill.BaseTestQuery;
+import org.junit.Test;
+
+public class TestHttpdPlugin extends BaseTestQuery {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestHttpdPlugin.class);
+
+  @Test
+  public void tryBasicQuery() throws Exception {
+    // test("select * from cp.`store/httpd/example1.httpd`");
+    test("select * from dfs.`${WORKING_PATH}/src/test/resources/store/httpd/example1.httpd`");
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/818f9450/exec/java-exec/src/test/resources/bootstrap-storage-plugins.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/bootstrap-storage-plugins.json b/exec/java-exec/src/test/resources/bootstrap-storage-plugins.json
index ec840a6..630db6b 100644
--- a/exec/java-exec/src/test/resources/bootstrap-storage-plugins.json
+++ b/exec/java-exec/src/test/resources/bootstrap-storage-plugins.json
@@ -40,6 +40,10 @@
         "json" : {
           type: "json"
         },
+        "httpd" :{
+          type: "httpd",
+          format: "%h %l %u %t \"%r\" %>s %b \"%{Referer}i\" \"%{User-Agent}i\" \"%{Cookie}i\""
+        },
         "txt" : {
           type : "text",
           extensions: [ "txt" ],

http://git-wip-us.apache.org/repos/asf/drill/blob/818f9450/exec/java-exec/src/test/resources/store/httpd/example1.httpd
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/store/httpd/example1.httpd b/exec/java-exec/src/test/resources/store/httpd/example1.httpd
new file mode 100644
index 0000000..531503b
--- /dev/null
+++ b/exec/java-exec/src/test/resources/store/httpd/example1.httpd
@@ -0,0 +1 @@
+2001:980:91c0:1:8d31:a232:25e5:85d - - [05/Sep/2010:11:27:50 +0200] "GET /b/ss/advbolprod2/1/H.22.1/s73176445413647?AQB=1&pccr=true&vidn=27F07A1B85012045-4000011500517C43&&ndh=1&t=19%2F5%2F2012%2023%3A51%3A27%202%20-120&ce=UTF-8&ns=bol&pageName=%2Fnl%2Fp%2Ffissler-speciaal-pannen-grillpan-28-x-28-cm%2F9200000002876066%2F&g=http%3A%2F%2Fwww.bol.com%2Fnl%2Fp%2Ffissler-speciaal-pannen-grillpan-28-x-28-cm%2F9200000002876066%2F%3Fpromo%3Dkoken-pannen_303_hs-koken-pannen-afj-120601_B3_product_1_9200000002876066%26bltg.pg_nm%3Dkoken-pannen%26bltg.slt_id%3D303%26bltg.slt_nm%3Dhs-koken-pannen-afj-120601%26bltg.slt_p&r=http%3A%2F%2Fwww.bol.com%2Fnl%2Fm%2Fkoken-tafelen%2Fkoken-pannen%2FN%2F11766%2Findex.html%3Fblabla%3Dblablawashere&cc=EUR&ch=D%3Dv3&server=ps316&events=prodView%2Cevent1%2Cevent2%2Cevent31&products=%3B9200000002876066%3B%3B%3B%3Bevar3%3Dkth%7Cevar8%3D9200000002876066_Fissler%20Speciaal%20Pannen%20-%20Grillpan%20-%2028%20x%2028%20cm%7Cevar35%3D170%7Cevar47%3DKTH%7Cevar9%3DNew%7C
 evar40%3Dno%20reviews%2C%3B%3B%3B%3Bevent31%3D423&c1=catalog%3Akth%3Aproduct-detail&v1=D%3Dc1&h1=catalog%2Fkth%2Fproduct-detail&h2=D%3DpageName&v3=kth&l3=endeca_001-mensen_default%2Cendeca_exact-boeken_default%2Cendeca_verschijningsjaar_default%2Cendeca_hardgoodscategoriesyn_default%2Cendeca_searchrank-hadoop_default%2Cendeca_genre_default%2Cendeca_uitvoering_default&v4=ps316&v6=koken-pannen_303_hs-koken-pannen-afj-120601_B3_product_1_9200000002876066&v10=Tu%2023%3A30&v12=logged%20in&v13=New&c25=niet%20ssl&c26=3631&c30=84.106.227.113.1323208998208762&v31=2000285551&c45=20120619235127&c46=20120501%204.3.4.1&c47=D%3Ds_vi&c49=%2Fnl%2Fcatalog%2Fproduct-detail.jsp&c50=%2Fnl%2Fcatalog%2Fproduct-detail.jsp&v51=www.bol.com&s=1280x800&c=24&j=1.7&v=N&k=Y&bw=1280&bh=272&p=Shockwave%20Flash%3B&AQE=1 HTTP/1.1" 200 23617 "http://www.google.nl/imgres?imgurl=http://daniel_en_sander.basjes.nl/fotos/geboorte-kaartje/geboortekaartje-binnenkant.jpg&imgrefurl=http://daniel_en_sander.basjes.nl/fotos/gebo
 orte-kaartje&usg=__LDxRMkacRs6yLluLcIrwoFsXY6o=&h=521&w=1024&sz=41&hl=nl&start=13&zoom=1&um=1&itbs=1&tbnid=Sqml3uGbjoyBYM:&tbnh=76&tbnw=150&prev=/images%3Fq%3Dbinnenkant%2Bgeboortekaartje%26um%3D1%26hl%3Dnl%26sa%3DN%26biw%3D1882%26bih%3D1014%26tbs%3Disch:1" "Mozilla/5.0 (Macintosh; U; Intel Mac OS X 10_6_4; nl-nl) AppleWebKit/533.17.8 (KHTML, like Gecko) Version/5.0.1 Safari/533.17.8" "jquery-ui-theme=Eggplant; BuI=SomeThing; Apache=127.0.0.1.1351111543699529"
\ No newline at end of file


[15/15] drill git commit: DRILL-1996: Add cancel method to Drill C++ connector

Posted by pa...@apache.org.
DRILL-1996: Add cancel method to Drill C++ connector

This closes #602


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

Branch: refs/heads/master
Commit: 83513daf0903e0d94fcaad7b1ae4e8ad6272b494
Parents: 166c4ce
Author: Laurent Goujon <la...@dremio.com>
Authored: Tue Oct 11 16:35:18 2016 -0700
Committer: Parth Chandra <pa...@apache.org>
Committed: Tue Nov 1 11:33:23 2016 -0700

----------------------------------------------------------------------
 .../native/client/example/querySubmitter.cpp    |   9 +-
 .../native/client/src/clientlib/drillClient.cpp |   8 +
 .../client/src/clientlib/drillClientImpl.cpp    | 174 +++++++++++--------
 .../client/src/clientlib/drillClientImpl.hpp    |   1 +
 .../client/src/include/drill/drillClient.hpp    |   7 +
 5 files changed, 124 insertions(+), 75 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/83513daf/contrib/native/client/example/querySubmitter.cpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/example/querySubmitter.cpp b/contrib/native/client/example/querySubmitter.cpp
index 306db56..2eeaf35 100644
--- a/contrib/native/client/example/querySubmitter.cpp
+++ b/contrib/native/client/example/querySubmitter.cpp
@@ -415,7 +415,14 @@ int main(int argc, char* argv[]) {
                     client.submitQuery(type, *queryInpIter, QueryResultsListener, NULL, &qryHandle);
                     client.registerSchemaChangeListener(&qryHandle, SchemaListener);
                     
-                    client.waitForResults();
+                     if(bTestCancel) {
+                        // Send cancellation request after 5seconds
+                        boost::this_thread::sleep(boost::posix_time::milliseconds(1000));
+                        std::cout<< "\n Cancelling query: " << *queryInpIter << "\n" << std::endl;
+                        client.cancelQuery(qryHandle);
+                    } else {
+                        client.waitForResults();
+                    }
 
                     client.freeQueryResources(&qryHandle);
                 }

http://git-wip-us.apache.org/repos/asf/drill/blob/83513daf/contrib/native/client/src/clientlib/drillClient.cpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/clientlib/drillClient.cpp b/contrib/native/client/src/clientlib/drillClient.cpp
index 20a466e..b02f993 100644
--- a/contrib/native/client/src/clientlib/drillClient.cpp
+++ b/contrib/native/client/src/clientlib/drillClient.cpp
@@ -400,6 +400,14 @@ status_t DrillClient::executeQuery(const PreparedStatement& pstmt, pfnQueryResul
 	return QRY_SUCCESS;
 }
 
+void DrillClient::cancelQuery(QueryHandle_t handle) {
+	if (!handle) {
+		return;
+	}
+	DrillClientQueryHandle* pHandle = static_cast<DrillClientQueryHandle*>(handle);
+	pHandle->cancel();
+}
+
 void* DrillClient::getApplicationContext(QueryHandle_t handle){
     assert(handle!=NULL);
     return (static_cast<DrillClientQueryHandle*>(handle))->getApplicationContext();

http://git-wip-us.apache.org/repos/asf/drill/blob/83513daf/contrib/native/client/src/clientlib/drillClientImpl.cpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/clientlib/drillClientImpl.cpp b/contrib/native/client/src/clientlib/drillClientImpl.cpp
index 7ecf910..51ae1a2 100644
--- a/contrib/native/client/src/clientlib/drillClientImpl.cpp
+++ b/contrib/native/client/src/clientlib/drillClientImpl.cpp
@@ -825,7 +825,6 @@ status_t DrillClientImpl::processQueryResult(AllocatedBufferPtr  allocatedBuffer
 status_t DrillClientImpl::processQueryData(AllocatedBufferPtr allocatedBuffer, const rpc::InBoundRpcMessage& msg ){
     DrillClientQueryResult* pDrillClientQueryResult=NULL;
     status_t ret=QRY_SUCCESS;
-    ::exec::shared::QueryId qid;
     // Be a good client and send ack as early as possible.
     // Drillbit pushed the query result to the client, the client should send ack
     // whenever it receives the message
@@ -839,7 +838,7 @@ status_t DrillClientImpl::processQueryData(AllocatedBufferPtr allocatedBuffer, c
         qr->ParseFromArray(msg.m_pbody.data(), msg.m_pbody.size());
         DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << qr->DebugString() << std::endl;)
 
-        qid = ::exec::shared::QueryId(qr->query_id());
+        const ::exec::shared::QueryId& qid = qr->query_id();
         if(qid.part1()==0){
             DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "DrillClientImpl::processQueryData: QID=0. Ignore and return QRY_SUCCESS." << std::endl;)
             delete allocatedBuffer;
@@ -855,90 +854,105 @@ status_t DrillClientImpl::processQueryData(AllocatedBufferPtr allocatedBuffer, c
             return ret;
         }
 
-        //Validate the RPC message
-        std::string valErr;
-        if( (ret=validateDataMessage(msg, *qr, valErr)) != QRY_SUCCESS){
-            delete allocatedBuffer;
-            delete qr;
-            DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "DrillClientImpl::processQueryData: ERR_QRY_INVRPC.\n";)
-            pDrillClientQueryResult->setQueryStatus(ret);
-            return handleQryError(ret, getMessage(ERR_QRY_INVRPC, valErr.c_str()), pDrillClientQueryResult);
-        }
-
-        //Build Record Batch here
-        DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << "Building record batch for Query Id - " << debugPrintQid(qr->query_id()) << std::endl;)
-
-        pRecordBatch= new RecordBatch(qr, allocatedBuffer,  msg.m_dbody);
-        pDrillClientQueryResult->m_numBatches++;
-
-        DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Allocated new Record batch." << (void*)pRecordBatch << std::endl;)
-        pRecordBatch->build();
-        DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << debugPrintQid(qr->query_id())<<"recordBatch.numRecords "
-            << pRecordBatch->getNumRecords()  << std::endl;)
-        DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << debugPrintQid(qr->query_id())<<"recordBatch.numFields "
-            << pRecordBatch->getNumFields()  << std::endl;)
-
-        ret=pDrillClientQueryResult->setupColumnDefs(qr);
-        if(ret==QRY_SUCCESS_WITH_INFO){
-            pRecordBatch->schemaChanged(true);
-        }
-
-        pDrillClientQueryResult->setIsQueryPending(true);
-        if(pDrillClientQueryResult->m_bIsLastChunk){
-            DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << debugPrintQid(*pDrillClientQueryResult->m_pQueryId)
-                <<  "Received last batch. " << std::endl;)
-            ret=QRY_NO_MORE_DATA;
+        // check if query has been cancelled
+        if (pDrillClientQueryResult->isCancelled()) {
+            DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << "Processing Query cancellation " << std::endl;)
+        	delete qr;
+        	delete allocatedBuffer;
+        	ret =  QRY_CANCEL;
+        } else {
+        	//Validate the RPC message
+        	std::string valErr;
+        	if( (ret=validateDataMessage(msg, *qr, valErr)) != QRY_SUCCESS){
+        		delete allocatedBuffer;
+        		delete qr;
+        		DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "DrillClientImpl::processQueryData: ERR_QRY_INVRPC.\n";)
+        		pDrillClientQueryResult->setQueryStatus(ret);
+        		return handleQryError(ret, getMessage(ERR_QRY_INVRPC, valErr.c_str()), pDrillClientQueryResult);
+        	}
+
+        	//Build Record Batch here
+        	DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << "Building record batch for Query Id - " << debugPrintQid(qid) << std::endl;)
+
+        	pRecordBatch= new RecordBatch(qr, allocatedBuffer,  msg.m_dbody);
+        	pDrillClientQueryResult->m_numBatches++;
+
+        	DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Allocated new Record batch." << (void*)pRecordBatch << std::endl;)
+        	pRecordBatch->build();
+        	DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << debugPrintQid(qid)<<"recordBatch.numRecords "
+        			<< pRecordBatch->getNumRecords()  << std::endl;)
+        	DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << debugPrintQid(qid)<<"recordBatch.numFields "
+        			<< pRecordBatch->getNumFields()  << std::endl;)
+
+					ret=pDrillClientQueryResult->setupColumnDefs(qr);
+        	if(ret==QRY_SUCCESS_WITH_INFO){
+        		pRecordBatch->schemaChanged(true);
+        	}
+
+        	pDrillClientQueryResult->setIsQueryPending(true);
+        	if(pDrillClientQueryResult->m_bIsLastChunk){
+        		DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << debugPrintQid(qid)
+        				<<  "Received last batch. " << std::endl;)
+            		ret=QRY_NO_MORE_DATA;
+        	}
+        	pDrillClientQueryResult->setQueryStatus(ret);
+        	ret = pDrillClientQueryResult->notifyListener(pRecordBatch, NULL);
         }
-        pDrillClientQueryResult->setQueryStatus(ret);
-        ret = pDrillClientQueryResult->notifyListener(pRecordBatch, NULL);
     } // release lock
-    if(ret==QRY_FAILURE){
-        sendCancel(&qid);
-        // Do not decrement pending requests here. We have sent a cancel and we may still receive results that are
-        // pushed on the wire before the cancel is processed.
-        pDrillClientQueryResult->setIsQueryPending(false);
-        DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << "Client app cancelled query." << std::endl;)
-        pDrillClientQueryResult->setQueryStatus(ret);
-        removeQueryHandle(pDrillClientQueryResult);
-        removeQueryResult(pDrillClientQueryResult);
-        return ret;
+    if((ret==QRY_FAILURE || ret==QRY_CANCELED) && pDrillClientQueryResult != NULL){
+        return handleQryCancellation(ret, pDrillClientQueryResult);
     }
     return ret;
 }
 
 status_t DrillClientImpl::processQueryId(AllocatedBufferPtr allocatedBuffer, const rpc::InBoundRpcMessage& msg ){
     DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << "Processing Query Handle with coordination id:" << msg.m_coord_id << std::endl;)
+	DrillClientQueryResult* pDrillClientQueryResult=NULL;
     status_t ret=QRY_SUCCESS;
 
     // make sure to deallocate buffer
     boost::shared_ptr<AllocatedBuffer> deallocationGuard(allocatedBuffer);
-    boost::lock_guard<boost::mutex> lock(m_dcMutex);
-    for(std::map< ::exec::shared::QueryId*, DrillClientQueryResult*>::const_iterator it=this->m_queryResults.begin();it!=this->m_queryResults.end();it++){
-        DrillClientQueryResult* pDrillClientQueryResult=it->second;
-        std::string qidString = (pDrillClientQueryResult->m_pQueryId!=NULL)?debugPrintQid(*pDrillClientQueryResult->m_pQueryId):std::string("NULL");
-        DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << "DrillClientImpl::processQueryId: m_queryIds: coordinationId: " << pDrillClientQueryResult->m_coordinationId
-        << " QueryId: "<< qidString << std::endl;)
-    }
-    if(msg.m_coord_id==0){
-        DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "DrillClientImpl::processQueryId: m_coord_id=0. Ignore and return QRY_SUCCESS." << std::endl;)
-        return QRY_SUCCESS;
-    }
-    std::map<int, DrillClientQueryHandle*>::const_iterator it;
-    it=this->m_queryHandles.find(msg.m_coord_id);
-    if(it!=this->m_queryHandles.end()){
-        DrillClientQueryResult* pDrillClientQueryResult=dynamic_cast<DrillClientQueryResult*>((*it).second);
-        if (!pDrillClientQueryResult) {
-            return handleQryError(QRY_INTERNAL_ERROR, getMessage(ERR_QRY_INVQUERYID), NULL);
-        }
-        exec::shared::QueryId *qid = new exec::shared::QueryId;
-        DRILL_MT_LOG(DRILL_LOG(LOG_TRACE)  << "Received Query Handle " << msg.m_pbody.size() << std::endl;)
-        qid->ParseFromArray(msg.m_pbody.data(), msg.m_pbody.size());
-        DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << "Query Id - " << debugPrintQid(*qid) << std::endl;)
-        m_queryResults[qid]=pDrillClientQueryResult;
-        //save queryId allocated here so we can free it later
-        pDrillClientQueryResult->setQueryId(qid);
-    }else{
-        return handleQryError(QRY_INTERNAL_ERROR, getMessage(ERR_QRY_INVQUERYID), NULL);
+    {
+    	boost::lock_guard<boost::mutex> lock(m_dcMutex);
+
+    	if(msg.m_coord_id==0){
+    		DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "DrillClientImpl::processQueryId: m_coord_id=0. Ignore and return QRY_SUCCESS." << std::endl;)
+      		return QRY_SUCCESS;
+    	}
+
+    	for(std::map< ::exec::shared::QueryId*, DrillClientQueryResult*>::const_iterator it=this->m_queryResults.begin();it!=this->m_queryResults.end();it++){
+    		DrillClientQueryResult* pQueryResult=it->second;
+    		std::string qidString = (pQueryResult->m_pQueryId!=NULL)?debugPrintQid(*pQueryResult->m_pQueryId):std::string("NULL");
+    		DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << "DrillClientImpl::processQueryId: m_queryIds: coordinationId: " << pQueryResult->m_coordinationId
+    				<< " QueryId: "<< qidString << std::endl;)
+    	}
+
+    	std::map<int, DrillClientQueryHandle*>::const_iterator it;
+    	it=this->m_queryHandles.find(msg.m_coord_id);
+    	if(it==this->m_queryHandles.end()){
+    		return handleQryError(QRY_INTERNAL_ERROR, getMessage(ERR_QRY_INVQUERYID), NULL);
+    	}
+    	pDrillClientQueryResult=dynamic_cast<DrillClientQueryResult*>((*it).second);
+    	if (!pDrillClientQueryResult) {
+    		return handleQryError(QRY_INTERNAL_ERROR, getMessage(ERR_QRY_INVQUERYID), NULL);
+    	}
+
+    	// Check for cancellation to notify
+    	if (pDrillClientQueryResult->isCancelled()) {
+    		ret = QRY_CANCELED;
+    	}
+    	else {
+    		exec::shared::QueryId *qid = new exec::shared::QueryId;
+    		DRILL_MT_LOG(DRILL_LOG(LOG_TRACE)  << "Received Query Handle " << msg.m_pbody.size() << std::endl;)
+    		qid->ParseFromArray(msg.m_pbody.data(), msg.m_pbody.size());
+    		DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << "Query Id - " << debugPrintQid(*qid) << std::endl;)
+    		m_queryResults[qid]=pDrillClientQueryResult;
+    		//save queryId allocated here so we can free it later
+    		pDrillClientQueryResult->setQueryId(qid);
+    	}
+    }
+    if (ret == QRY_CANCELED && pDrillClientQueryResult != NULL) {
+    	return handleQryCancellation(ret, pDrillClientQueryResult);
     }
     return ret;
 }
@@ -1486,6 +1500,18 @@ status_t DrillClientImpl::handleQryError(status_t status,
     return status;
 }
 
+status_t DrillClientImpl::handleQryCancellation(status_t status, DrillClientQueryResult* pQueryHandle) {
+	sendCancel(&pQueryHandle->getQueryId());
+	// Do not decrement pending requests here. We have sent a cancel and we may still receive results that are
+	// pushed on the wire before the cancel is processed.
+	pQueryHandle->setIsQueryPending(false);
+	DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << "Client app cancelled query." << std::endl;)
+	pQueryHandle->setQueryStatus(status);
+	removeQueryResult(pQueryHandle);
+	removeQueryHandle(pQueryHandle);
+	return status;
+}
+
 void DrillClientImpl::broadcastError(DrillClientError* pErr){
     if(pErr!=NULL){
         std::map<int, DrillClientQueryHandle*>::const_iterator iter;

http://git-wip-us.apache.org/repos/asf/drill/blob/83513daf/contrib/native/client/src/clientlib/drillClientImpl.hpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/clientlib/drillClientImpl.hpp b/contrib/native/client/src/clientlib/drillClientImpl.hpp
index f9d0779..8da37b6 100644
--- a/contrib/native/client/src/clientlib/drillClientImpl.hpp
+++ b/contrib/native/client/src/clientlib/drillClientImpl.hpp
@@ -489,6 +489,7 @@ class DrillClientImpl : public DrillClientImplBase{
         status_t validateDataMessage(const rpc::InBoundRpcMessage& msg, const exec::shared::QueryData& qd, std::string& valError);
         status_t validateResultMessage(const rpc::InBoundRpcMessage& msg, const exec::shared::QueryResult& qr, std::string& valError);
         connectionStatus_t handleConnError(connectionStatus_t status, const std::string& msg);
+        status_t handleQryCancellation(status_t status, DrillClientQueryResult* pQueryResult);
         status_t handleQryError(status_t status, const std::string& msg, DrillClientQueryHandle* pQueryHandle);
         status_t handleQryError(status_t status, const exec::shared::DrillPBError& e, DrillClientQueryHandle* pQueryHandle);
         // handle query state indicating query is COMPLETED or CANCELED

http://git-wip-us.apache.org/repos/asf/drill/blob/83513daf/contrib/native/client/src/include/drill/drillClient.hpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/include/drill/drillClient.hpp b/contrib/native/client/src/include/drill/drillClient.hpp
index 5e59885..29ae6c2 100644
--- a/contrib/native/client/src/include/drill/drillClient.hpp
+++ b/contrib/native/client/src/include/drill/drillClient.hpp
@@ -1276,6 +1276,13 @@ class DECLSPEC_DRILL_CLIENT DrillClient{
         status_t executeQuery(const PreparedStatement& pstmt, pfnQueryResultsListener listener, void* listenerCtx, QueryHandle_t* qHandle);
 
         /*
+         * Cancel a query.
+         *
+         * @param[in] the handle of the query to cancel
+         */
+        void cancelQuery(QueryHandle_t handle);
+
+        /*
          * The client application should call this function to wait for results if it has registered a
          * listener.
          */


[02/15] drill git commit: DRILL-3423: Adding HTTPd Log Parsing functionality including full pushdown, type remapping and wildcard support. Pushed through the requested columns for push down to the parser. Added more tests to cover a few more use cases.

Posted by pa...@apache.org.
DRILL-3423: Adding HTTPd Log Parsing functionality including full pushdown, type remapping and wildcard support.
 Pushed through the requested columns for push down to the parser. Added more tests to cover a few more use cases. Ensured that user query fields are now completely consistent with returned values.


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

Branch: refs/heads/master
Commit: 46c0f2a4135450417dfebf52f11538f8926fd467
Parents: 818f945
Author: Jim Scott <ji...@13ways.com>
Authored: Tue Nov 3 15:43:54 2015 -0600
Committer: Parth Chandra <pa...@apache.org>
Committed: Tue Nov 1 10:42:22 2016 -0700

----------------------------------------------------------------------
 exec/java-exec/pom.xml                          |   3 +-
 .../exec/store/dfs/easy/EasyFormatPlugin.java   |   1 +
 .../exec/store/httpd/HttpdLogFormatPlugin.java  | 246 +++++++++++++++
 .../drill/exec/store/httpd/HttpdLogRecord.java  | 299 +++++++++++++++++++
 .../drill/exec/store/httpd/HttpdParser.java     | 171 +++++++++++
 .../resources/bootstrap-storage-plugins.json    |   5 +-
 .../java/org/apache/drill/BaseTestQuery.java    |  76 ++++-
 .../drill/exec/store/httpd/HttpdParserTest.java |  48 +++
 .../store/httpd/TestHttpdLogFormatPlugin.java   |  97 ++++++
 .../drill/exec/store/text/TestTextColumn.java   | 103 ++-----
 .../resources/bootstrap-storage-plugins.json    |   4 +-
 .../resources/store/httpd/dfs-bootstrap.httpd   |   2 +
 .../store/httpd/dfs-test-bootstrap-test.httpd   |   5 +
 pom.xml                                         |   1 +
 14 files changed, 976 insertions(+), 85 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/46c0f2a4/exec/java-exec/pom.xml
----------------------------------------------------------------------
diff --git a/exec/java-exec/pom.xml b/exec/java-exec/pom.xml
index e9b07be..6bf77c0 100644
--- a/exec/java-exec/pom.xml
+++ b/exec/java-exec/pom.xml
@@ -456,9 +456,8 @@
     <dependency>
       <groupId>nl.basjes.parse.httpdlog</groupId>
       <artifactId>httpdlog-parser</artifactId>
-      <version>2.1.1</version>
+      <version>2.4</version>
     </dependency>
-        
   </dependencies>
 
   <profiles>

http://git-wip-us.apache.org/repos/asf/drill/blob/46c0f2a4/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
index c7f9cf8..c09e009 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
@@ -117,6 +117,7 @@ public abstract class EasyFormatPlugin<T extends FormatPluginConfig> implements
 
   /** Method indicates whether or not this format could also be in a compression container (for example: csv.gz versus csv).
    * If this format uses its own internal compression scheme, such as Parquet does, then this should return false.
+   * @return <code>true</code> if it is compressible
    */
   public boolean isCompressible() {
     return compressible;

http://git-wip-us.apache.org/repos/asf/drill/blob/46c0f2a4/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdLogFormatPlugin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdLogFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdLogFormatPlugin.java
new file mode 100644
index 0000000..7a83d45
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdLogFormatPlugin.java
@@ -0,0 +1,246 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.drill.exec.store.httpd;
+
+import java.io.IOException;
+import java.util.List;
+
+import nl.basjes.parse.core.exceptions.DissectionFailure;
+import nl.basjes.parse.core.exceptions.InvalidDissectorException;
+import nl.basjes.parse.core.exceptions.MissingDissectorsException;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.logical.FormatPluginConfig;
+import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.physical.impl.OutputMutator;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.store.AbstractRecordReader;
+import org.apache.drill.exec.store.RecordWriter;
+import org.apache.drill.exec.store.dfs.DrillFileSystem;
+import org.apache.drill.exec.store.dfs.easy.EasyFormatPlugin;
+import org.apache.drill.exec.store.dfs.easy.EasyWriter;
+import org.apache.drill.exec.store.dfs.easy.FileWork;
+import org.apache.drill.exec.vector.complex.impl.VectorContainerWriter;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter.ComplexWriter;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.LineRecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.TextInputFormat;
+
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import java.util.Map;
+import org.apache.drill.exec.store.RecordReader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class HttpdLogFormatPlugin extends EasyFormatPlugin<HttpdLogFormatPlugin.HttpdLogFormatConfig> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(HttpdLogFormatPlugin.class);
+  private static final String PLUGIN_EXTENSION = "httpd";
+  private static final int VECTOR_MEMORY_ALLOCATION = 4095;
+
+  public HttpdLogFormatPlugin(final String name, final DrillbitContext context, final Configuration fsConf,
+      final StoragePluginConfig storageConfig, final HttpdLogFormatConfig formatConfig) {
+
+    super(name, context, fsConf, storageConfig, formatConfig, true, false, true, true,
+        Lists.newArrayList(PLUGIN_EXTENSION), PLUGIN_EXTENSION);
+  }
+
+  /**
+   * This class is a POJO to hold the configuration for the HttpdLogFormat Parser. This is automatically
+   * serialized/deserialized from JSON format.
+   */
+  @JsonTypeName(PLUGIN_EXTENSION)
+  public static class HttpdLogFormatConfig implements FormatPluginConfig {
+
+    private String logFormat;
+    private String timestampFormat;
+
+    /**
+     * @return the logFormat
+     */
+    public String getLogFormat() {
+      return logFormat;
+    }
+
+    /**
+     * @return the timestampFormat
+     */
+    public String getTimestampFormat() {
+      return timestampFormat;
+    }
+  }
+
+  /**
+   * This class performs the work for the plugin. This is where all logic goes to read records. In this case httpd logs
+   * are lines terminated with a new line character.
+   */
+  private class HttpdLogRecordReader extends AbstractRecordReader {
+
+    private final DrillFileSystem fs;
+    private final FileWork work;
+    private final FragmentContext fragmentContext;
+    private ComplexWriter writer;
+    private HttpdParser parser;
+    private LineRecordReader lineReader;
+    private LongWritable lineNumber;
+
+    public HttpdLogRecordReader(final FragmentContext context, final DrillFileSystem fs, final FileWork work, final List<SchemaPath> columns) {
+      this.fs = fs;
+      this.work = work;
+      this.fragmentContext = context;
+      setColumns(columns);
+    }
+
+    /**
+     * The query fields passed in are formatted in a way that Drill requires. Those must be cleaned up to work with the
+     * parser.
+     *
+     * @return Map<DrillFieldNames, ParserFieldNames>
+     */
+    private Map<String, String> makeParserFields() {
+      final Map<String, String> fieldMapping = Maps.newHashMap();
+      for (final SchemaPath sp : getColumns()) {
+        final String drillField = sp.getRootSegment().getPath();
+        final String parserField = HttpdParser.parserFormattedFieldName(drillField);
+        fieldMapping.put(drillField, parserField);
+      }
+      return fieldMapping;
+    }
+
+    @Override
+    public void setup(final OperatorContext context, final OutputMutator output) throws ExecutionSetupException {
+      try {
+        /**
+         * Extract the list of field names for the parser to use if it is NOT a star query. If it is a star query just
+         * pass through an empty map, because the parser is going to have to build all possibilities.
+         */
+        final Map<String, String> fieldMapping = !isStarQuery() ? makeParserFields() : null;
+        writer = new VectorContainerWriter(output);
+        parser = new HttpdParser(writer.rootAsMap(), context.getManagedBuffer(),
+            HttpdLogFormatPlugin.this.getConfig().getLogFormat(),
+            HttpdLogFormatPlugin.this.getConfig().getTimestampFormat(),
+            fieldMapping);
+
+        final Path path = fs.makeQualified(new Path(work.getPath()));
+        FileSplit split = new FileSplit(path, work.getStart(), work.getLength(), new String[]{""});
+        TextInputFormat inputFormat = new TextInputFormat();
+        JobConf job = new JobConf(fs.getConf());
+        job.setInt("io.file.buffer.size", fragmentContext.getConfig().getInt(ExecConstants.TEXT_LINE_READER_BUFFER_SIZE));
+        job.setInputFormat(inputFormat.getClass());
+        lineReader = (LineRecordReader) inputFormat.getRecordReader(split, job, Reporter.NULL);
+        lineNumber = lineReader.createKey();
+      }
+      catch (NoSuchMethodException | MissingDissectorsException | InvalidDissectorException e) {
+        throw handleAndGenerate("Failure creating HttpdParser", e);
+      }
+      catch (IOException e) {
+        throw handleAndGenerate("Failure creating HttpdRecordReader", e);
+      }
+    }
+
+    private RuntimeException handleAndGenerate(final String s, final Exception e) {
+      throw UserException.dataReadError(e)
+          .message(s + "\n%s", e.getMessage())
+          .addContext("Path", work.getPath())
+          .addContext("Split Start", work.getStart())
+          .addContext("Split Length", work.getLength())
+          .addContext("Local Line Number", lineNumber.get())
+          .build(LOG);
+    }
+
+    /**
+     * This record reader is given a batch of records (lines) to read. Next acts upon a batch of records.
+     *
+     * @return Number of records in this batch.
+     */
+    @Override
+    public int next() {
+      try {
+        final Text line = lineReader.createValue();
+
+        writer.allocate();
+        writer.reset();
+
+        int recordCount = 0;
+        while (recordCount < VECTOR_MEMORY_ALLOCATION && lineReader.next(lineNumber, line)) {
+          writer.setPosition(recordCount);
+          parser.parse(line.toString());
+          recordCount++;
+        }
+        writer.setValueCount(recordCount);
+
+        return recordCount;
+      }
+      catch (DissectionFailure | InvalidDissectorException | MissingDissectorsException | IOException e) {
+        throw handleAndGenerate("Failure while parsing log record.", e);
+      }
+    }
+
+    @Override
+    public void close() throws Exception {
+      try {
+        if (lineReader != null) {
+          lineReader.close();
+        }
+      }
+      catch (IOException e) {
+        LOG.warn("Failure while closing Httpd reader.", e);
+      }
+    }
+
+  }
+
+  /**
+   * This plugin supports pushing down into the parser. Only fields specifically asked for within the configuration will
+   * be parsed. If no fields are asked for then all possible fields will be returned.
+   *
+   * @return true
+   */
+  @Override
+  public boolean supportsPushDown() {
+    return true;
+  }
+
+  @Override
+  public RecordReader getRecordReader(final FragmentContext context, final DrillFileSystem dfs, final FileWork fileWork, final List<SchemaPath> columns, final String userName) throws ExecutionSetupException {
+    return new HttpdLogRecordReader(context, dfs, fileWork, columns);
+  }
+
+  @Override
+  public RecordWriter getRecordWriter(final FragmentContext context, final EasyWriter writer) throws IOException {
+    throw new UnsupportedOperationException("Drill doesn't currently support writing HTTPd logs");
+  }
+
+  @Override
+  public int getReaderOperatorType() {
+    return -1;
+  }
+
+  @Override
+  public int getWriterOperatorType() {
+    return -1;
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/46c0f2a4/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdLogRecord.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdLogRecord.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdLogRecord.java
new file mode 100644
index 0000000..03f70c1
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdLogRecord.java
@@ -0,0 +1,299 @@
+/*
+ * Copyright 2015 The Apache Software Foundation.
+ *
+ * Licensed 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.drill.exec.store.httpd;
+
+import com.google.common.base.Charsets;
+import com.google.common.collect.Maps;
+import io.netty.buffer.DrillBuf;
+import java.util.EnumSet;
+import java.util.Map;
+import nl.basjes.parse.core.Casts;
+import nl.basjes.parse.core.Parser;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter.MapWriter;
+import org.apache.drill.exec.vector.complex.writer.BigIntWriter;
+import org.apache.drill.exec.vector.complex.writer.Float8Writer;
+import org.apache.drill.exec.vector.complex.writer.VarCharWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class HttpdLogRecord {
+
+  private static final Logger LOG = LoggerFactory.getLogger(HttpdLogRecord.class);
+  private final Map<String, VarCharWriter> strings = Maps.newHashMap();
+  private final Map<String, BigIntWriter> longs = Maps.newHashMap();
+  private final Map<String, Float8Writer> doubles = Maps.newHashMap();
+  private final Map<String, MapWriter> wildcards = Maps.newHashMap();
+  private final Map<String, String> cleanExtensions = Maps.newHashMap();
+  private final Map<String, MapWriter> startedWildcards = Maps.newHashMap();
+  private final Map<String, MapWriter> wildcardWriters = Maps.newHashMap();
+  private DrillBuf managedBuffer;
+
+  public HttpdLogRecord(final DrillBuf managedBuffer) {
+    this.managedBuffer = managedBuffer;
+  }
+
+  /**
+   * Call this method after a record has been parsed. This finished the lifecycle of any maps that were written and
+   * removes all the entries for the next record to be able to work.
+   */
+  public void finishRecord() {
+    for (MapWriter writer : wildcardWriters.values()) {
+      writer.end();
+    }
+    wildcardWriters.clear();
+    startedWildcards.clear();
+  }
+
+  private DrillBuf buf(final int size) {
+    if (managedBuffer.capacity() < size) {
+      managedBuffer = managedBuffer.reallocIfNeeded(size);
+    }
+    return managedBuffer;
+  }
+
+  private void writeString(final VarCharWriter writer, final String value) {
+    final byte[] stringBytes = value.getBytes(Charsets.UTF_8);
+    final DrillBuf stringBuffer = buf(stringBytes.length);
+    stringBuffer.clear();
+    stringBuffer.writeBytes(stringBytes);
+    writer.writeVarChar(0, stringBytes.length, stringBuffer);
+  }
+
+  /**
+   * This method is referenced and called via reflection. This is added as a parsing target for the parser. It will get
+   * called when the value of a log field is a String data type.
+   *
+   * @param field name of field
+   * @param value value of field
+   */
+  @SuppressWarnings("unused")
+  public void set(final String field, final String value) {
+    if (value != null) {
+      final VarCharWriter w = strings.get(field);
+      if (w != null) {
+        LOG.trace("Parsed field: {}, as string: {}", field, value);
+        writeString(w, value);
+      }
+      else {
+        LOG.warn("No 'string' writer found for field: {}", field);
+      }
+    }
+  }
+
+  /**
+   * This method is referenced and called via reflection. This is added as a parsing target for the parser. It will get
+   * called when the value of a log field is a Long data type.
+   *
+   * @param field name of field
+   * @param value value of field
+   */
+  @SuppressWarnings("unused")
+  public void set(final String field, final Long value) {
+    if (value != null) {
+      final BigIntWriter w = longs.get(field);
+      if (w != null) {
+        LOG.trace("Parsed field: {}, as long: {}", field, value);
+        w.writeBigInt(value);
+      }
+      else {
+        LOG.warn("No 'long' writer found for field: {}", field);
+      }
+    }
+  }
+
+  /**
+   * This method is referenced and called via reflection. This is added as a parsing target for the parser. It will get
+   * called when the value of a log field is a Double data type.
+   *
+   * @param field name of field
+   * @param value value of field
+   */
+  @SuppressWarnings("unused")
+  public void set(final String field, final Double value) {
+    if (value != null) {
+      final Float8Writer w = doubles.get(field);
+      if (w != null) {
+        LOG.trace("Parsed field: {}, as double: {}", field, value);
+        w.writeFloat8(value);
+      }
+      else {
+        LOG.warn("No 'double' writer found for field: {}", field);
+      }
+    }
+  }
+
+  /**
+   * This method is referenced and called via reflection. When the parser processes a field like:
+   * HTTP.URI:request.firstline.uri.query.* where star is an arbitrary field that the parser found this method will be
+   * invoked. <br>
+   *
+   * @param field name of field
+   * @param value value of field
+   */
+  @SuppressWarnings("unused")
+  public void setWildcard(final String field, final String value) {
+    if (value != null) {
+      final MapWriter mapWriter = getWildcardWriter(field);
+      LOG.trace("Parsed wildcard field: {}, as string: {}", field, value);
+      final VarCharWriter w = mapWriter.varChar(cleanExtensions.get(field));
+      writeString(w, value);
+    }
+  }
+
+  /**
+   * This method is referenced and called via reflection. When the parser processes a field like:
+   * HTTP.URI:request.firstline.uri.query.* where star is an arbitrary field that the parser found this method will be
+   * invoked. <br>
+   *
+   * @param field name of field
+   * @param value value of field
+   */
+  @SuppressWarnings("unused")
+  public void setWildcard(final String field, final Long value) {
+    if (value != null) {
+      final MapWriter mapWriter = getWildcardWriter(field);
+      LOG.trace("Parsed wildcard field: {}, as long: {}", field, value);
+      final BigIntWriter w = mapWriter.bigInt(cleanExtensions.get(field));
+      w.writeBigInt(value);
+    }
+  }
+
+  /**
+   * This method is referenced and called via reflection. When the parser processes a field like:
+   * HTTP.URI:request.firstline.uri.query.* where star is an arbitrary field that the parser found this method will be
+   * invoked. <br>
+   *
+   * @param field name of field
+   * @param value value of field
+   */
+  @SuppressWarnings("unused")
+  public void setWildcard(final String field, final Double value) {
+    if (value != null) {
+      final MapWriter mapWriter = getWildcardWriter(field);
+      LOG.trace("Parsed wildcard field: {}, as double: {}", field, value);
+      final Float8Writer w = mapWriter.float8(cleanExtensions.get(field));
+      w.writeFloat8(value);
+    }
+  }
+
+  /**
+   * For a configuration like HTTP.URI:request.firstline.uri.query.*, a writer was created with name
+   * HTTP.URI:request.firstline.uri.query, we traverse the list of wildcard writers to see which one is the root of the
+   * name of the field passed in like HTTP.URI:request.firstline.uri.query.old. This is writer entry that is needed.
+   *
+   * @param field like HTTP.URI:request.firstline.uri.query.old where 'old' is one of many different parameter names.
+   * @return the writer to be used for this field.
+   */
+  private MapWriter getWildcardWriter(final String field) {
+    MapWriter writer = startedWildcards.get(field);
+    if (writer == null) {
+      for (Map.Entry<String, MapWriter> entry : wildcards.entrySet()) {
+        final String root = entry.getKey();
+        if (field.startsWith(root)) {
+          writer = entry.getValue();
+
+          /**
+           * In order to save some time, store the cleaned version of the field extension. It is possible it will have
+           * unsafe characters in it.
+           */
+          if (!cleanExtensions.containsKey(field)) {
+            final String extension = field.substring(root.length() + 1, field.length());
+            final String cleanExtension = HttpdParser.drillFormattedFieldName(extension);
+            cleanExtensions.put(field, cleanExtension);
+            LOG.debug("Added extension: field='{}' with cleanExtension='{}'", field, cleanExtension);
+          }
+
+          /**
+           * We already know we have the writer, but if we have put this writer in the started list, do NOT call start
+           * again.
+           */
+          if (!wildcardWriters.containsKey(root)) {
+            /**
+             * Start and store this root map writer for later retrieval.
+             */
+            LOG.debug("Starting new wildcard field writer: {}", field);
+            writer.start();
+            startedWildcards.put(field, writer);
+            wildcardWriters.put(root, writer);
+          }
+
+          /**
+           * Break out of the for loop when we find a root writer that matches the field.
+           */
+          break;
+        }
+      }
+    }
+
+    return writer;
+  }
+
+  public Map<String, VarCharWriter> getStrings() {
+    return strings;
+  }
+
+  public Map<String, BigIntWriter> getLongs() {
+    return longs;
+  }
+
+  public Map<String, Float8Writer> getDoubles() {
+    return doubles;
+  }
+
+  /**
+   * This record will be used with a single parser. For each field that is to be parsed a setter will be called. It
+   * registers a setter method for each field being parsed. It also builds the data writers to hold the data beings
+   * parsed.
+   *
+   * @param parser
+   * @param mapWriter
+   * @param type
+   * @param parserFieldName
+   * @param drillFieldName
+   * @throws NoSuchMethodException
+   */
+  public void addField(final Parser<HttpdLogRecord> parser, final MapWriter mapWriter, final EnumSet<Casts> type, final String parserFieldName, final String drillFieldName) throws NoSuchMethodException {
+    final boolean hasWildcard = parserFieldName.endsWith(HttpdParser.PARSER_WILDCARD);
+
+    /**
+     * This is a dynamic way to map the setter for each specified field type. <br/>
+     * e.g. a TIME.STAMP may map to a LONG while a referrer may map to a STRING
+     */
+    if (hasWildcard) {
+      final String cleanName = parserFieldName.substring(0, parserFieldName.length() - HttpdParser.PARSER_WILDCARD.length());
+      LOG.debug("Adding WILDCARD parse target: {} as {}, with field name: {}", parserFieldName, cleanName, drillFieldName);
+      parser.addParseTarget(this.getClass().getMethod("setWildcard", String.class, String.class), parserFieldName);
+      parser.addParseTarget(this.getClass().getMethod("setWildcard", String.class, Double.class), parserFieldName);
+      parser.addParseTarget(this.getClass().getMethod("setWildcard", String.class, Long.class), parserFieldName);
+      wildcards.put(cleanName, mapWriter.map(drillFieldName));
+    }
+    else if (type.contains(Casts.DOUBLE)) {
+      LOG.debug("Adding DOUBLE parse target: {}, with field name: {}", parserFieldName, drillFieldName);
+      parser.addParseTarget(this.getClass().getMethod("set", String.class, Double.class), parserFieldName);
+      doubles.put(parserFieldName, mapWriter.float8(drillFieldName));
+    }
+    else if (type.contains(Casts.LONG)) {
+      LOG.debug("Adding LONG parse target: {}, with field name: {}", parserFieldName, drillFieldName);
+      parser.addParseTarget(this.getClass().getMethod("set", String.class, Long.class), parserFieldName);
+      longs.put(parserFieldName, mapWriter.bigInt(drillFieldName));
+    }
+    else {
+      LOG.debug("Adding STRING parse target: {}, with field name: {}", parserFieldName, drillFieldName);
+      parser.addParseTarget(this.getClass().getMethod("set", String.class, String.class), parserFieldName);
+      strings.put(parserFieldName, mapWriter.varChar(drillFieldName));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/46c0f2a4/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdParser.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdParser.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdParser.java
new file mode 100644
index 0000000..114a7f4
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdParser.java
@@ -0,0 +1,171 @@
+/*
+ * Copyright 2015 The Apache Software Foundation.
+ *
+ * Licensed 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.drill.exec.store.httpd;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Maps;
+import io.netty.buffer.DrillBuf;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import nl.basjes.parse.core.Casts;
+import nl.basjes.parse.core.Parser;
+import nl.basjes.parse.core.exceptions.DissectionFailure;
+import nl.basjes.parse.core.exceptions.InvalidDissectorException;
+import nl.basjes.parse.core.exceptions.MissingDissectorsException;
+import nl.basjes.parse.httpdlog.HttpdLoglineParser;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter.MapWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class HttpdParser {
+
+  private static final Logger LOG = LoggerFactory.getLogger(HttpdParser.class);
+  public static final String PARSER_WILDCARD = ".*";
+  public static final String SAFE_WILDCARD = "_$";
+  public static final String SAFE_SEPARATOR = "_";
+  public static final String REMAPPING_FLAG = "#";
+  private final Parser<HttpdLogRecord> parser;
+  private final HttpdLogRecord record;
+
+  public HttpdParser(final MapWriter mapWriter, final DrillBuf managedBuffer, final String logFormat,
+      final String timestampFormat, final Map<String, String> fieldMapping)
+      throws NoSuchMethodException, MissingDissectorsException, InvalidDissectorException {
+
+    Preconditions.checkArgument(logFormat != null && !logFormat.trim().isEmpty(), "logFormat cannot be null or empty");
+
+    this.record = new HttpdLogRecord(managedBuffer);
+    this.parser = new HttpdLoglineParser<>(HttpdLogRecord.class, logFormat, timestampFormat);
+
+    setupParser(mapWriter, logFormat, fieldMapping);
+
+    if (timestampFormat != null && !timestampFormat.trim().isEmpty()) {
+      LOG.info("Custom timestamp format has been specified. This is an informational note only as custom timestamps is rather unusual.");
+    }
+    if (logFormat.contains("\n")) {
+      LOG.info("Specified logformat is a multiline log format: {}", logFormat);
+    }
+  }
+
+  /**
+   * We do not expose the underlying parser or the record which is used to manage the writers.
+   *
+   * @param line log line to tear apart.
+   *
+   * @throws DissectionFailure
+   * @throws InvalidDissectorException
+   * @throws MissingDissectorsException
+   */
+  public void parse(final String line) throws DissectionFailure, InvalidDissectorException, MissingDissectorsException {
+    parser.parse(record, line);
+    record.finishRecord();
+  }
+
+  /**
+   * In order to define a type remapping the format of the field configuration will look like: <br/>
+   * HTTP.URI:request.firstline.uri.query.[parameter name] <br/>
+   *
+   * @param parser Add type remapping to this parser instance.
+   * @param fieldName request.firstline.uri.query.[parameter_name]
+   * @param fieldType HTTP.URI, etc..
+   */
+  private void addTypeRemapping(final Parser<HttpdLogRecord> parser, final String fieldName, final String fieldType) {
+    LOG.debug("Adding type remapping - fieldName: {}, fieldType: {}", fieldName, fieldType);
+    parser.addTypeRemapping(fieldName, fieldType);
+  }
+
+  /**
+   * The parser deals with dots unlike Drill wanting underscores request_referer. For the sake of simplicity we are
+   * going replace the dots. The resultant output field will look like: request.referer.<br>
+   * Additionally, wild cards will get replaced with .*
+   *
+   * @param drillFieldName name to be cleansed.
+   * @return
+   */
+  public static String parserFormattedFieldName(final String drillFieldName) {
+    return drillFieldName.replace(SAFE_WILDCARD, PARSER_WILDCARD).replaceAll(SAFE_SEPARATOR, ".").replaceAll("\\.\\.", "_");
+  }
+
+  /**
+   * Drill cannot deal with fields with dots in them like request.referer. For the sake of simplicity we are going
+   * ensure the field name is cleansed. The resultant output field will look like: request_referer.<br>
+   * Additionally, wild cards will get replaced with _$
+   *
+   * @param parserFieldName name to be cleansed.
+   * @return
+   */
+  public static String drillFormattedFieldName(final String parserFieldName) {
+    return parserFieldName.replaceAll("_", "__").replace(PARSER_WILDCARD, SAFE_WILDCARD).replaceAll("\\.", SAFE_SEPARATOR);
+  }
+
+  private void setupParser(final MapWriter mapWriter, final String logFormat, final Map<String, String> fieldMapping)
+      throws NoSuchMethodException, MissingDissectorsException, InvalidDissectorException {
+
+    /**
+     * If the user has selected fields, then we will use them to configure the parser because this would be the most
+     * efficient way to parse the log.
+     */
+    final Map<String, String> requestedPaths;
+    final List<String> allParserPaths = parser.getPossiblePaths();
+    if (fieldMapping != null && !fieldMapping.isEmpty()) {
+      LOG.debug("Using fields defined by user.");
+      requestedPaths = fieldMapping;
+    }
+    else {
+      /**
+       * Use all possible paths that the parser has determined from the specified log format.
+       */
+      LOG.debug("No fields defined by user, defaulting to all possible fields.");
+      requestedPaths = Maps.newHashMap();
+      for (final String parserPath : allParserPaths) {
+        requestedPaths.put(drillFormattedFieldName(parserPath), parserPath);
+      }
+    }
+
+    /**
+     * By adding the parse target to the dummy instance we activate it for use. Which we can then use to find out which
+     * paths cast to which native data types. After we are done figuring this information out, we throw this away
+     * because this will be the slowest parsing path possible for the specified format.
+     */
+    Parser<Object> dummy = new HttpdLoglineParser<>(Object.class, logFormat);
+    dummy.addParseTarget(String.class.getMethod("indexOf", String.class), allParserPaths);
+
+    for (final Map.Entry<String, String> entry : requestedPaths.entrySet()) {
+      final EnumSet<Casts> casts;
+
+      /**
+       * Check the field specified by the user to see if it is supposed to be remapped.
+       */
+      if (entry.getValue().startsWith(REMAPPING_FLAG)) {
+        /**
+         * Because this field is being remapped we need to replace the field name that the parser uses.
+         */
+        entry.setValue(entry.getValue().substring(REMAPPING_FLAG.length()));
+
+        final String[] pieces = entry.getValue().split(":");
+        addTypeRemapping(parser, pieces[1], pieces[0]);
+
+        casts = Casts.STRING_ONLY;
+      }
+      else {
+        casts = dummy.getCasts(entry.getValue());
+      }
+
+      LOG.debug("Setting up drill field: {}, parser field: {}, which casts as: {}", entry.getKey(), entry.getValue(), casts);
+      record.addField(parser, mapWriter, casts, entry.getValue(), entry.getKey());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/46c0f2a4/exec/java-exec/src/main/resources/bootstrap-storage-plugins.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/bootstrap-storage-plugins.json b/exec/java-exec/src/main/resources/bootstrap-storage-plugins.json
index dab4ada..b5485d2 100644
--- a/exec/java-exec/src/main/resources/bootstrap-storage-plugins.json
+++ b/exec/java-exec/src/main/resources/bootstrap-storage-plugins.json
@@ -29,9 +29,10 @@
           extensions: [ "tsv" ],
           delimiter: "\t"
         },
-        "httpd" :{
+        "httpd" : {
           type: "httpd",
-          format: "%h %l %u %t \"%r\" %>s %b \"%{Referer}i\" \"%{User-Agent}i\" \"%{Cookie}i\""
+          logFormat: "%h %t \"%r\" %>s %b \"%{Referer}i\""
+          /* timestampFormat: "dd/MMM/yyyy:HH:mm:ss ZZ" */
         },
         "parquet" : {
           type: "parquet"

http://git-wip-us.apache.org/repos/asf/drill/blob/46c0f2a4/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java b/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
index e528d0e..93916e9 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
@@ -18,6 +18,7 @@
 package org.apache.drill;
 
 import static org.hamcrest.core.StringContains.containsString;
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.fail;
 
@@ -65,6 +66,10 @@ import org.junit.runner.Description;
 import com.google.common.base.Charsets;
 import com.google.common.base.Preconditions;
 import com.google.common.io.Resources;
+import java.util.ArrayList;
+import java.util.Arrays;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.vector.ValueVector;
 
 public class BaseTestQuery extends ExecTest {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BaseTestQuery.class);
@@ -302,7 +307,7 @@ public class BaseTestQuery extends ExecTest {
       Preconditions.checkArgument(query instanceof String, "Expected a string as input query");
       query = QueryTestUtil.normalizeQuery((String)query);
       return client.runQuery(type, (String)query);
-    }
+  }
   }
 
   public static List<QueryDataBatch> testPreparedStatement(PreparedStatementHandle handle) throws Exception {
@@ -384,9 +389,9 @@ public class BaseTestQuery extends ExecTest {
       } catch (AssertionError e) {
         e.addSuppressed(actualException);
         throw e;
-      }
     }
   }
+  }
 
   /**
    * Utility method which tests given query produces a {@link UserException}
@@ -501,4 +506,69 @@ public class BaseTestQuery extends ExecTest {
 
     return formattedResults.toString();
   }
-}
+
+
+  public class TestResultSet {
+
+    private final List<List<String>> rows;
+
+    public TestResultSet() {
+      rows = new ArrayList<>();
+    }
+
+    public TestResultSet(List<QueryDataBatch> batches) throws SchemaChangeException {
+      rows = new ArrayList<>();
+      convert(batches);
+    }
+
+    public void addRow(String... cells) {
+      List<String> newRow = Arrays.asList(cells);
+      rows.add(newRow);
+    }
+
+    public int size() {
+      return rows.size();
+    }
+
+    @Override public boolean equals(Object o) {
+      boolean result = false;
+
+      if (this == o) {
+        result = true;
+      } else if (o instanceof TestResultSet) {
+        TestResultSet that = (TestResultSet) o;
+        assertEquals(this.size(), that.size());
+        for (int i = 0; i < this.rows.size(); i++) {
+          assertEquals(this.rows.get(i).size(), that.rows.get(i).size());
+          for (int j = 0; j < this.rows.get(i).size(); ++j) {
+            assertEquals(this.rows.get(i).get(j), that.rows.get(i).get(j));
+          }
+        }
+        result = true;
+      }
+
+      return result;
+    }
+
+    private void convert(List<QueryDataBatch> batches) throws SchemaChangeException {
+      RecordBatchLoader loader = new RecordBatchLoader(getAllocator());
+      for (QueryDataBatch batch : batches) {
+        int rc = batch.getHeader().getRowCount();
+        if (batch.getData() != null) {
+          loader.load(batch.getHeader().getDef(), batch.getData());
+          for (int i = 0; i < rc; ++i) {
+            List<String> newRow = new ArrayList<>();
+            rows.add(newRow);
+            for (VectorWrapper<?> vw : loader) {
+              ValueVector.Accessor accessor = vw.getValueVector().getAccessor();
+              Object o = accessor.getObject(i);
+              newRow.add(o == null ? null : o.toString());
+            }
+          }
+        }
+        loader.clear();
+        batch.release();
+      }
+    }
+  }
+ }

http://git-wip-us.apache.org/repos/asf/drill/blob/46c0f2a4/exec/java-exec/src/test/java/org/apache/drill/exec/store/httpd/HttpdParserTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/httpd/HttpdParserTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/httpd/HttpdParserTest.java
new file mode 100644
index 0000000..961d9a6
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/httpd/HttpdParserTest.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2015 The Apache Software Foundation.
+ *
+ * Licensed 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.drill.exec.store.httpd;
+
+import io.netty.buffer.DrillBuf;
+import java.util.Map;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter.MapWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class HttpdParserTest {
+
+  private static final Logger LOG = LoggerFactory.getLogger(HttpdParserTest.class);
+
+  private void runTest(String logFormat, String logLine) throws Exception {
+    MapWriter mapWriter = null;
+    DrillBuf managedBuffer = null;
+    Map<String, String> configuredFields = null;
+    HttpdParser parser = new HttpdParser(mapWriter, managedBuffer, logFormat, null, configuredFields);
+    parser.parse(logLine);
+  }
+
+//  @Test
+  public void testFirstPattern() throws Exception {
+    LOG.info("testFirstPattern");
+//    final String format = "common";
+//    final String format = "%h %l %u %t \"%r\" %>s %b";
+    final String format = "%h %t \"%r\" %>s %b \"%{Referer}i\"";
+    final String line = "127.0.0.1 [31/Dec/2012:23:49:41 +0100] "
+        + "\"GET /foo HTTP/1.1\" 200 "
+        + "1213 \"http://localhost/index.php?mies=wim\"";
+    runTest(format, line);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/46c0f2a4/exec/java-exec/src/test/java/org/apache/drill/exec/store/httpd/TestHttpdLogFormatPlugin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/httpd/TestHttpdLogFormatPlugin.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/httpd/TestHttpdLogFormatPlugin.java
new file mode 100644
index 0000000..6c2eaf8
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/httpd/TestHttpdLogFormatPlugin.java
@@ -0,0 +1,97 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.drill.exec.store.httpd;
+
+import java.util.List;
+import org.apache.drill.BaseTestQuery;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
+import static org.junit.Assert.*;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TestHttpdLogFormatPlugin extends BaseTestQuery {
+
+  private static final Logger LOG = LoggerFactory.getLogger(TestHttpdLogFormatPlugin.class);
+
+  /**
+   * This test covers the test bootstrap-storage-plugins.json section of httpd.
+   *
+   * Indirectly this validates the HttpdLogFormatPlugin.HttpdLogFormatConfig deserializing properly.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testDfsTestBootstrap_star() throws Exception {
+    test("select * from dfs_test.`${WORKING_PATH}/src/test/resources/store/httpd/dfs-test-bootstrap-test.httpd`");
+  }
+
+  /**
+   * This test covers the test bootstrap-storage-plugins.json section of httpd.
+   *
+   * Indirectly this validates the HttpdLogFormatPlugin.HttpdLogFormatConfig deserializing properly.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testDfsTestBootstrap_notstar() throws Exception {
+    test("select `TIME_STAMP:request_receive_time`, `HTTP_METHOD:request_firstline_method`, `STRING:request_status_last`, `BYTES:response_body_bytesclf` \n"
+        + "from dfs_test.`${WORKING_PATH}/src/test/resources/store/httpd/dfs-test-bootstrap-test.httpd`");
+  }
+
+  /**
+   * This test covers the main bootstrap-storage-plugins.json section of httpd.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testDfsBootstrap_star() throws Exception {
+    test("select * from dfs.`${WORKING_PATH}/src/test/resources/store/httpd/dfs-bootstrap.httpd`");
+  }
+
+  /**
+   * This test covers the main bootstrap-storage-plugins.json section of httpd.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testDfsBootstrap_wildcard() throws Exception {
+    test("select `STRING:request_referer_query_$` from dfs.`${WORKING_PATH}/src/test/resources/store/httpd/dfs-bootstrap.httpd`");
+  }
+
+  /**
+   * This test covers the main bootstrap-storage-plugins.json section of httpd.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testDfsBootstrap_underscore() throws Exception {
+    test("select `TIME_DAY:request_receive_time_day__utc` from dfs.`${WORKING_PATH}/src/test/resources/store/httpd/dfs-bootstrap.httpd`");
+  }
+
+  @Test
+  public void testGroupBy_1() throws Exception {
+    final List<QueryDataBatch> actualResults = testSqlWithResults(
+        "select `HTTP_METHOD:request_firstline_method` as http_method, `STRING:request_status_last` as status_code, sum(`BYTES:response_body_bytesclf`) as total_bytes \n"
+        + "from dfs_test.`${WORKING_PATH}/src/test/resources/store/httpd/dfs-test-bootstrap-test.httpd`\n"
+        + "group by `HTTP_METHOD:request_firstline_method`, `STRING:request_status_last`"
+    );
+
+    final TestResultSet expectedResultSet = new TestResultSet();
+    expectedResultSet.addRow("GET", "200", "46551");
+    expectedResultSet.addRow("POST", "302", "18186");
+
+    TestResultSet actualResultSet = new TestResultSet(actualResults);
+    assertTrue(expectedResultSet.equals(actualResultSet));
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/46c0f2a4/exec/java-exec/src/test/java/org/apache/drill/exec/store/text/TestTextColumn.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/text/TestTextColumn.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/text/TestTextColumn.java
index 882033a..1ff6818 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/text/TestTextColumn.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/text/TestTextColumn.java
@@ -1,107 +1,58 @@
 /**
- * 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
+ * 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.
+ * 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.drill.exec.store.text;
 
-import static org.junit.Assert.assertEquals;
-
-import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.List;
 
 import org.apache.drill.BaseTestQuery;
-import org.apache.drill.exec.exception.SchemaChangeException;
-import org.apache.drill.exec.record.RecordBatchLoader;
-import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.rpc.user.QueryDataBatch;
-import org.apache.drill.exec.vector.ValueVector;
 import org.junit.Test;
+import static org.junit.Assert.*;
+
+public class TestTextColumn extends BaseTestQuery {
 
-public class TestTextColumn extends BaseTestQuery{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestTextColumn.class);
 
   @Test
-  public void testCsvColumnSelection() throws Exception{
+  public void testCsvColumnSelection() throws Exception {
     test("select columns[0] as region_id, columns[1] as country from dfs_test.`[WORKING_PATH]/src/test/resources/store/text/data/regions.csv`");
   }
 
   @Test
   public void testDefaultDelimiterColumnSelection() throws Exception {
-    List<QueryDataBatch> batches = testSqlWithResults("SELECT columns[0] as entire_row " +
-      "from dfs_test.`[WORKING_PATH]/src/test/resources/store/text/data/letters.txt`");
+    List<QueryDataBatch> actualResults = testSqlWithResults("SELECT columns[0] as entire_row "
+        + "from dfs_test.`[WORKING_PATH]/src/test/resources/store/text/data/letters.txt`");
 
-    List<List<String>> expectedOutput = Arrays.asList(
-      Arrays.asList("a, b,\",\"c\",\"d,, \\n e"),
-      Arrays.asList("d, e,\",\"f\",\"g,, \\n h"),
-      Arrays.asList("g, h,\",\"i\",\"j,, \\n k"));
+    final TestResultSet expectedResultSet = new TestResultSet();
+    expectedResultSet.addRow("a, b,\",\"c\",\"d,, \\n e");
+    expectedResultSet.addRow("d, e,\",\"f\",\"g,, \\n h");
+    expectedResultSet.addRow("g, h,\",\"i\",\"j,, \\n k");
 
-    List<List<String>> actualOutput = getOutput(batches);
-    System.out.println(actualOutput);
-    validateOutput(expectedOutput, actualOutput);
+    TestResultSet actualResultSet = new TestResultSet(actualResults);
+    assertTrue(expectedResultSet.equals(actualResultSet));
   }
 
   @Test
   public void testCsvColumnSelectionCommasInsideQuotes() throws Exception {
-    List<QueryDataBatch> batches = testSqlWithResults("SELECT columns[0] as col1, columns[1] as col2, columns[2] as col3," +
-      "columns[3] as col4 from dfs_test.`[WORKING_PATH]/src/test/resources/store/text/data/letters.csv`");
+    List<QueryDataBatch> actualResults = testSqlWithResults("SELECT columns[0] as col1, columns[1] as col2, columns[2] as col3,"
+        + "columns[3] as col4 from dfs_test.`[WORKING_PATH]/src/test/resources/store/text/data/letters.csv`");
 
-    List<List<String>> expectedOutput = Arrays.asList(
-      Arrays.asList("a, b,", "c", "d,, \\n e","f\\\"g"),
-      Arrays.asList("d, e,", "f", "g,, \\n h","i\\\"j"),
-      Arrays.asList("g, h,", "i", "j,, \\n k","l\\\"m"));
-
-    List<List<String>> actualOutput = getOutput(batches);
-    validateOutput(expectedOutput, actualOutput);
-  }
+    final TestResultSet expectedResultSet = new TestResultSet();
+    expectedResultSet.addRow("a, b,", "c", "d,, \\n e", "f\\\"g");
+    expectedResultSet.addRow("d, e,", "f", "g,, \\n h", "i\\\"j");
+    expectedResultSet.addRow("g, h,", "i", "j,, \\n k", "l\\\"m");
 
-  private List<List<String>> getOutput(List<QueryDataBatch> batches) throws SchemaChangeException {
-    List<List<String>> output = new ArrayList<>();
-    RecordBatchLoader loader = new RecordBatchLoader(getAllocator());
-    int last = 0;
-    for(QueryDataBatch batch : batches) {
-      int rows = batch.getHeader().getRowCount();
-      if(batch.getData() != null) {
-        loader.load(batch.getHeader().getDef(), batch.getData());
-        // TODO:  Clean:  DRILL-2933:  That load(...) no longer throws
-        // SchemaChangeException, so check/clean throws clause above.
-        for (int i = 0; i < rows; ++i) {
-          output.add(new ArrayList<String>());
-          for (VectorWrapper<?> vw: loader) {
-            ValueVector.Accessor accessor = vw.getValueVector().getAccessor();
-            Object o = accessor.getObject(i);
-            output.get(last).add(o == null ? null: o.toString());
-          }
-          ++last;
-        }
-      }
-      loader.clear();
-      batch.release();
-    }
-    return output;
+    TestResultSet actualResultSet = new TestResultSet(actualResults);
+    assertTrue(expectedResultSet.equals(actualResultSet));
   }
-
-  private void validateOutput(List<List<String>> expected, List<List<String>> actual) {
-    assertEquals(expected.size(), actual.size());
-    for (int i = 0 ; i < expected.size(); ++i) {
-      assertEquals(expected.get(i).size(), actual.get(i).size());
-      for (int j = 0; j < expected.get(i).size(); ++j) {
-        assertEquals(expected.get(i).get(j), actual.get(i).get(j));
-      }
-    }
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/46c0f2a4/exec/java-exec/src/test/resources/bootstrap-storage-plugins.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/bootstrap-storage-plugins.json b/exec/java-exec/src/test/resources/bootstrap-storage-plugins.json
index 630db6b..7b977e2 100644
--- a/exec/java-exec/src/test/resources/bootstrap-storage-plugins.json
+++ b/exec/java-exec/src/test/resources/bootstrap-storage-plugins.json
@@ -40,9 +40,9 @@
         "json" : {
           type: "json"
         },
-        "httpd" :{
+        "httpd" : {
           type: "httpd",
-          format: "%h %l %u %t \"%r\" %>s %b \"%{Referer}i\" \"%{User-Agent}i\" \"%{Cookie}i\""
+          logFormat: "%h %l %u %t \"%r\" %>s %b \"%{Referer}i\" \"%{User-agent}i\""
         },
         "txt" : {
           type : "text",

http://git-wip-us.apache.org/repos/asf/drill/blob/46c0f2a4/exec/java-exec/src/test/resources/store/httpd/dfs-bootstrap.httpd
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/store/httpd/dfs-bootstrap.httpd b/exec/java-exec/src/test/resources/store/httpd/dfs-bootstrap.httpd
new file mode 100644
index 0000000..fd12566
--- /dev/null
+++ b/exec/java-exec/src/test/resources/store/httpd/dfs-bootstrap.httpd
@@ -0,0 +1,2 @@
+127.0.0.1 [31/Dec/2012:23:49:41 +0100] "GET /foo HTTP/1.1" 200 1213 "http://localhost/index.php?mies=wim&blue=red"
+127.0.0.1 [31/Dec/2012:23:49:41 +0100] "GET /foo HTTP/1.1" 200 1213 "http://localhost/index.php?mies=wim&test=true"

http://git-wip-us.apache.org/repos/asf/drill/blob/46c0f2a4/exec/java-exec/src/test/resources/store/httpd/dfs-test-bootstrap-test.httpd
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/store/httpd/dfs-test-bootstrap-test.httpd b/exec/java-exec/src/test/resources/store/httpd/dfs-test-bootstrap-test.httpd
new file mode 100644
index 0000000..d48fa12
--- /dev/null
+++ b/exec/java-exec/src/test/resources/store/httpd/dfs-test-bootstrap-test.httpd
@@ -0,0 +1,5 @@
+195.154.46.135 - - [25/Oct/2015:04:11:25 +0100] "GET /linux/doing-pxe-without-dhcp-control HTTP/1.1" 200 24323 "http://howto.basjes.nl/" "Mozilla/5.0 (Windows NT 5.1; rv:35.0) Gecko/20100101 Firefox/35.0"
+23.95.237.180 - - [25/Oct/2015:04:11:26 +0100] "GET /join_form HTTP/1.0" 200 11114 "http://howto.basjes.nl/" "Mozilla/5.0 (Windows NT 5.1; rv:35.0) Gecko/20100101 Firefox/35.0"
+23.95.237.180 - - [25/Oct/2015:04:11:27 +0100] "POST /join_form HTTP/1.1" 302 9093 "http://howto.basjes.nl/join_form" "Mozilla/5.0 (Windows NT 5.1; rv:35.0) Gecko/20100101 Firefox/35.0"
+158.222.5.157 - - [25/Oct/2015:04:24:31 +0100] "GET /join_form HTTP/1.0" 200 11114 "http://howto.basjes.nl/" "Mozilla/5.0 (Windows NT 6.3; WOW64; rv:34.0) Gecko/20100101 Firefox/34.0 AlexaToolbar/alxf-2.21"
+158.222.5.157 - - [25/Oct/2015:04:24:32 +0100] "POST /join_form HTTP/1.1" 302 9093 "http://howto.basjes.nl/join_form" "Mozilla/5.0 (Windows NT 6.3; WOW64; rv:34.0) Gecko/20100101 Firefox/34.0 AlexaToolbar/alxf-2.21"

http://git-wip-us.apache.org/repos/asf/drill/blob/46c0f2a4/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index dcb1b3d..e094c4b 100644
--- a/pom.xml
+++ b/pom.xml
@@ -223,6 +223,7 @@
             <exclude>**/*.linux</exclude>
             <exclude>**/client/build/**</exclude>
             <exclude>**/*.tbl</exclude>
+            <exclude>**/*.httpd</exclude>
             <!-- TODO DRILL-4336: try to avoid the need to add this -->
             <exclude>dependency-reduced-pom.xml</exclude>
           </excludes>


[04/15] drill git commit: DRILL-4373: Drill and Hive have incompatible timestamp representations in parquet - added sys/sess option "store.parquet.int96_as_timestamp"; - added int96 to timestamp converter for both readers; - added unit tests;

Posted by pa...@apache.org.
DRILL-4373: Drill and Hive have incompatible timestamp representations in parquet - added sys/sess option "store.parquet.int96_as_timestamp"; - added int96 to timestamp converter for both readers; - added unit tests;

This closes #600


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

Branch: refs/heads/master
Commit: 7e7214b40784668d1599f265067f789aedb6cf86
Parents: 4a82bc1
Author: Vitalii Diravka <vi...@gmail.com>
Authored: Fri Sep 2 21:43:50 2016 +0000
Committer: Parth Chandra <pa...@apache.org>
Committed: Tue Nov 1 10:43:06 2016 -0700

----------------------------------------------------------------------
 ...onvertHiveParquetScanToDrillParquetScan.java |   3 +-
 .../exec/store/hive/HiveStoragePlugin.java      |   2 +
 .../org/apache/drill/exec/ExecConstants.java    |   2 +
 .../impl/conv/ConvertFromImpalaTimestamp.java   |  35 ++++++---
 .../server/options/SystemOptionManager.java     |   1 +
 .../store/parquet/ParquetReaderUtility.java     |  39 ++++++++--
 .../columnreaders/ColumnReaderFactory.java      |  15 +++-
 .../NullableFixedByteAlignedReaders.java        |  28 +++++++
 .../columnreaders/ParquetRecordReader.java      |   4 +
 .../ParquetToDrillTypeConverter.java            |   7 +-
 .../parquet2/DrillParquetGroupConverter.java    |  32 +++++++-
 .../test/java/org/apache/drill/TestBuilder.java |  11 +++
 .../physical/impl/writer/TestParquetWriter.java |  76 +++++++++++++++++--
 .../parquet/int96_dict_change/000000_0          | Bin 0 -> 270 bytes
 .../parquet/int96_dict_change/000000_1          | Bin 0 -> 312 bytes
 .../testInt96DictChange/q1.tsv                  |  12 +++
 16 files changed, 240 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/7e7214b4/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/logical/ConvertHiveParquetScanToDrillParquetScan.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/logical/ConvertHiveParquetScanToDrillParquetScan.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/logical/ConvertHiveParquetScanToDrillParquetScan.java
index c43664c..228308f 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/logical/ConvertHiveParquetScanToDrillParquetScan.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/logical/ConvertHiveParquetScanToDrillParquetScan.java
@@ -68,7 +68,7 @@ public class ConvertHiveParquetScanToDrillParquetScan extends StoragePluginOptim
   public static final ConvertHiveParquetScanToDrillParquetScan INSTANCE = new ConvertHiveParquetScanToDrillParquetScan();
 
   private static final DrillSqlOperator INT96_TO_TIMESTAMP =
-      new DrillSqlOperator("convert_fromTIMESTAMP_IMPALA", 1, true);
+      new DrillSqlOperator("convert_fromTIMESTAMP_IMPALA_LOCALTIMEZONE", 1, true);
 
   private static final DrillSqlOperator RTRIM = new DrillSqlOperator("RTRIM", 1, true);
 
@@ -296,6 +296,7 @@ public class ConvertHiveParquetScanToDrillParquetScan extends StoragePluginOptim
     if (outputType.getSqlTypeName() == SqlTypeName.TIMESTAMP) {
       // TIMESTAMP is stored as INT96 by Hive in ParquetFormat. Use convert_fromTIMESTAMP_IMPALA UDF to convert
       // INT96 format data to TIMESTAMP
+      // TODO: Remove this conversion once "store.parquet.reader.int96_as_timestamp" will be true by default
       return rb.makeCall(INT96_TO_TIMESTAMP, inputRef);
     }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/7e7214b4/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveStoragePlugin.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveStoragePlugin.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveStoragePlugin.java
index 8f8fdba..f99a934 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveStoragePlugin.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveStoragePlugin.java
@@ -114,6 +114,8 @@ public class HiveStoragePlugin extends AbstractStoragePlugin {
 
   @Override
   public Set<StoragePluginOptimizerRule> getPhysicalOptimizerRules(OptimizerRulesContext optimizerRulesContext) {
+    // TODO: Remove implicit using of convert_fromTIMESTAMP_IMPALA function
+    // once "store.parquet.reader.int96_as_timestamp" will be true by default
     if(optimizerRulesContext.getPlannerSettings().getOptions()
         .getOption(ExecConstants.HIVE_OPTIMIZE_SCAN_WITH_NATIVE_READERS).bool_val) {
       return ImmutableSet.<StoragePluginOptimizerRule>of(ConvertHiveParquetScanToDrillParquetScan.INSTANCE);

http://git-wip-us.apache.org/repos/asf/drill/blob/7e7214b4/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
index 053311f..21015bb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
@@ -144,6 +144,8 @@ public interface ExecConstants {
   OptionValidator PARQUET_VECTOR_FILL_CHECK_THRESHOLD_VALIDATOR = new PositiveLongValidator(PARQUET_VECTOR_FILL_CHECK_THRESHOLD, 100l, 10l);
   String PARQUET_NEW_RECORD_READER = "store.parquet.use_new_reader";
   OptionValidator PARQUET_RECORD_READER_IMPLEMENTATION_VALIDATOR = new BooleanValidator(PARQUET_NEW_RECORD_READER, false);
+  String PARQUET_READER_INT96_AS_TIMESTAMP = "store.parquet.reader.int96_as_timestamp";
+  OptionValidator PARQUET_READER_INT96_AS_TIMESTAMP_VALIDATOR = new BooleanValidator(PARQUET_READER_INT96_AS_TIMESTAMP, false);
 
   OptionValidator COMPILE_SCALAR_REPLACEMENT = new BooleanValidator("exec.compile.scalar_replacement", false);
 

http://git-wip-us.apache.org/repos/asf/drill/blob/7e7214b4/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/ConvertFromImpalaTimestamp.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/ConvertFromImpalaTimestamp.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/ConvertFromImpalaTimestamp.java
index a57eede..38e0514 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/ConvertFromImpalaTimestamp.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/ConvertFromImpalaTimestamp.java
@@ -28,6 +28,29 @@ import org.apache.drill.exec.expr.holders.VarBinaryHolder;
 public class ConvertFromImpalaTimestamp {
 
 
+  @FunctionTemplate(name = "convert_fromTIMESTAMP_IMPALA_LOCALTIMEZONE", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class ImpalaTimestampConvertFromWithLocalTimezone implements DrillSimpleFunc {
+
+    @Param VarBinaryHolder in;
+    @Output TimeStampHolder out;
+
+
+    @Override
+    public void setup() { }
+
+    @Override
+    public void eval() {
+      org.apache.drill.exec.util.ByteBufUtil.checkBufferLength(in.buffer, in.start, in.end, 12);
+
+      in.buffer.readerIndex(in.start);
+      long nanosOfDay = in.buffer.readLong();
+      int julianDay = in.buffer.readInt();
+      long dateTime = (julianDay - org.apache.drill.exec.store.parquet.ParquetReaderUtility.JULIAN_DAY_NUMBER_FOR_UNIX_EPOCH) *
+          org.joda.time.DateTimeConstants.MILLIS_PER_DAY + (nanosOfDay / org.apache.drill.exec.store.parquet.ParquetReaderUtility.NanoTimeUtils.NANOS_PER_MILLISECOND);
+      out.value = new org.joda.time.DateTime(dateTime, org.joda.time.chrono.JulianChronology.getInstance()).withZoneRetainFields(org.joda.time.DateTimeZone.UTC).getMillis();
+    }
+  }
+
   @FunctionTemplate(name = "convert_fromTIMESTAMP_IMPALA", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
   public static class ImpalaTimestampConvertFrom implements DrillSimpleFunc {
 
@@ -45,16 +68,8 @@ public class ConvertFromImpalaTimestamp {
       in.buffer.readerIndex(in.start);
       long nanosOfDay = in.buffer.readLong();
       int julianDay = in.buffer.readInt();
-      /* We use the same implementation as org.joda.time.DateTimeUtils.fromJulianDay but avoid rounding errors
-         Note we need to subtract half of a day because julian days are recorded as starting at noon.
-         From Joda :
-              public static final long fromJulianDay(double julianDay) {
-                484            double epochDay = julianDay - 2440587.5d;
-                485            return (long) (epochDay * 86400000d);
-                486        }
-      */
-      long dateTime = (julianDay - 2440588)*86400000L + (nanosOfDay / 1000000);
-      out.value = new org.joda.time.DateTime((long) dateTime, org.joda.time.chrono.JulianChronology.getInstance()).withZoneRetainFields(org.joda.time.DateTimeZone.UTC).getMillis();
+      out.value = (julianDay - org.apache.drill.exec.store.parquet.ParquetReaderUtility.JULIAN_DAY_NUMBER_FOR_UNIX_EPOCH) *
+          org.joda.time.DateTimeConstants.MILLIS_PER_DAY + (nanosOfDay / org.apache.drill.exec.store.parquet.ParquetReaderUtility.NanoTimeUtils.NANOS_PER_MILLISECOND);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/7e7214b4/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
index 71ebd7d..f272c9d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
@@ -100,6 +100,7 @@ public class SystemOptionManager extends BaseOptionManager implements AutoClosea
       ExecConstants.PARQUET_VECTOR_FILL_THRESHOLD_VALIDATOR,
       ExecConstants.PARQUET_VECTOR_FILL_CHECK_THRESHOLD_VALIDATOR,
       ExecConstants.PARQUET_RECORD_READER_IMPLEMENTATION_VALIDATOR,
+      ExecConstants.PARQUET_READER_INT96_AS_TIMESTAMP_VALIDATOR,
       ExecConstants.JSON_READER_ALL_TEXT_MODE_VALIDATOR,
       ExecConstants.ENABLE_UNION_TYPE,
       ExecConstants.TEXT_ESTIMATED_ROW_SIZE,

http://git-wip-us.apache.org/repos/asf/drill/blob/7e7214b4/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetReaderUtility.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetReaderUtility.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetReaderUtility.java
index 1f6dc1e..470cc00 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetReaderUtility.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetReaderUtility.java
@@ -39,6 +39,8 @@ import org.apache.parquet.hadoop.metadata.ParquetMetadata;
 import org.apache.parquet.schema.OriginalType;
 import org.joda.time.Chronology;
 import org.joda.time.DateTimeConstants;
+import org.apache.parquet.example.data.simple.NanoTime;
+import org.apache.parquet.io.api.Binary;
 
 import java.util.Arrays;
 import java.util.HashMap;
@@ -76,21 +78,21 @@ public class ParquetReaderUtility {
    * in the data pages themselves to see if they are likely corrupt.
    */
   public enum DateCorruptionStatus {
-    META_SHOWS_CORRUPTION{
+    META_SHOWS_CORRUPTION {
       @Override
-      public String toString(){
+      public String toString() {
         return "It is determined from metadata that the date values are definitely CORRUPT";
       }
     },
     META_SHOWS_NO_CORRUPTION {
       @Override
-      public String toString(){
+      public String toString() {
         return "It is determined from metadata that the date values are definitely CORRECT";
       }
     },
     META_UNCLEAR_TEST_VALUES {
       @Override
-      public String toString(){
+      public String toString() {
         return "Not enough info in metadata, parquet reader will test individual date values";
       }
     }
@@ -152,7 +154,7 @@ public class ParquetReaderUtility {
             OriginalType originalType = columnMetadata.getOriginalType();
             if (OriginalType.DATE.equals(originalType) && columnMetadata.hasSingleValue() &&
                 (Integer) columnMetadata.getMaxValue() > ParquetReaderUtility.DATE_CORRUPTION_THRESHOLD) {
-              int newMinMax = ParquetReaderUtility.autoCorrectCorruptedDate((Integer)columnMetadata.getMaxValue());
+              int newMinMax = ParquetReaderUtility.autoCorrectCorruptedDate((Integer) columnMetadata.getMaxValue());
               columnMetadata.setMax(newMinMax);
               columnMetadata.setMin(newMinMax);
             }
@@ -290,4 +292,31 @@ public class ParquetReaderUtility {
     }
     return DateCorruptionStatus.META_SHOWS_NO_CORRUPTION;
   }
+
+  /**
+   * Utilities for converting from parquet INT96 binary (impala, hive timestamp)
+   * to date time value. This utilizes the Joda library.
+   */
+  public static class NanoTimeUtils {
+
+    public static final long NANOS_PER_MILLISECOND = 1000000;
+
+  /**
+   * @param binaryTimeStampValue
+   *          hive, impala timestamp values with nanoseconds precision
+   *          are stored in parquet Binary as INT96 (12 constant bytes)
+   *
+   * @return  Unix Timestamp - the number of milliseconds since January 1, 1970, 00:00:00 GMT
+   *          represented by @param binaryTimeStampValue .
+   */
+    public static long getDateTimeValueFromBinary(Binary binaryTimeStampValue) {
+      // This method represents binaryTimeStampValue as ByteBuffer, where timestamp is stored as sum of
+      // julian day number (32-bit) and nanos of day (64-bit)
+      NanoTime nt = NanoTime.fromBinary(binaryTimeStampValue);
+      int julianDay = nt.getJulianDay();
+      long nanosOfDay = nt.getTimeOfDayNanos();
+      return (julianDay - JULIAN_DAY_NUMBER_FOR_UNIX_EPOCH) * DateTimeConstants.MILLIS_PER_DAY
+          + nanosOfDay / NANOS_PER_MILLISECOND;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/7e7214b4/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ColumnReaderFactory.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ColumnReaderFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ColumnReaderFactory.java
index ea65615..662d5c9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ColumnReaderFactory.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ColumnReaderFactory.java
@@ -18,6 +18,7 @@
 package org.apache.drill.exec.store.parquet.columnreaders;
 
 import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.vector.BigIntVector;
 import org.apache.drill.exec.vector.BitVector;
@@ -241,7 +242,12 @@ public class ColumnReaderFactory {
 
     if (! columnChunkMetaData.getEncodings().contains(Encoding.PLAIN_DICTIONARY)) {
       if (columnDescriptor.getType() == PrimitiveType.PrimitiveTypeName.INT96) {
-        return new NullableFixedByteAlignedReaders.NullableFixedBinaryReader(parentReader, allocateSize, columnDescriptor, columnChunkMetaData, true, (NullableVarBinaryVector) valueVec, schemaElement);
+         // TODO: check convertedType once parquet support TIMESTAMP_NANOS type annotation.
+        if (parentReader.getFragmentContext().getOptions().getOption(ExecConstants.PARQUET_READER_INT96_AS_TIMESTAMP).bool_val) {
+          return new NullableFixedByteAlignedReaders.NullableFixedBinaryAsTimeStampReader(parentReader, allocateSize, columnDescriptor, columnChunkMetaData, true, (NullableTimeStampVector) valueVec, schemaElement);
+        } else {
+          return new NullableFixedByteAlignedReaders.NullableFixedBinaryReader(parentReader, allocateSize, columnDescriptor, columnChunkMetaData, true, (NullableVarBinaryVector) valueVec, schemaElement);
+        }
       }else{
         return new NullableFixedByteAlignedReaders.NullableFixedByteAlignedReader<>(parentReader, allocateSize, columnDescriptor, columnChunkMetaData, fixedLength, valueVec, schemaElement);
       }
@@ -272,7 +278,12 @@ public class ColumnReaderFactory {
               throw new ExecutionSetupException("Unsupported nullable converted type " + convertedType + " for primitive type INT64");
           }
         case INT96:
-           return new NullableFixedByteAlignedReaders.NullableFixedBinaryReader(parentReader, allocateSize, columnDescriptor, columnChunkMetaData, true, (NullableVarBinaryVector) valueVec, schemaElement);
+          // TODO: check convertedType once parquet support TIMESTAMP_NANOS type annotation.
+          if (parentReader.getFragmentContext().getOptions().getOption(ExecConstants.PARQUET_READER_INT96_AS_TIMESTAMP).bool_val) {
+            return new NullableFixedByteAlignedReaders.NullableFixedBinaryAsTimeStampReader(parentReader, allocateSize, columnDescriptor, columnChunkMetaData, true, (NullableTimeStampVector) valueVec, schemaElement);
+          } else {
+            return new NullableFixedByteAlignedReaders.NullableFixedBinaryReader(parentReader, allocateSize, columnDescriptor, columnChunkMetaData, true, (NullableVarBinaryVector) valueVec, schemaElement);
+          }
         case FLOAT:
           return new NullableFixedByteAlignedReaders.NullableDictionaryFloat4Reader(parentReader, allocateSize, columnDescriptor, columnChunkMetaData, fixedLength, (NullableFloat4Vector)valueVec, schemaElement);
         case DOUBLE:

http://git-wip-us.apache.org/repos/asf/drill/blob/7e7214b4/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/NullableFixedByteAlignedReaders.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/NullableFixedByteAlignedReaders.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/NullableFixedByteAlignedReaders.java
index df4c1ba..f4fe5ee 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/NullableFixedByteAlignedReaders.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/NullableFixedByteAlignedReaders.java
@@ -46,6 +46,7 @@ import org.apache.parquet.io.api.Binary;
 import org.joda.time.DateTimeConstants;
 
 import io.netty.buffer.DrillBuf;
+import static org.apache.drill.exec.store.parquet.ParquetReaderUtility.NanoTimeUtils.getDateTimeValueFromBinary;
 
 public class NullableFixedByteAlignedReaders {
 
@@ -107,6 +108,33 @@ public class NullableFixedByteAlignedReaders {
     }
   }
 
+  /**
+   * Class for reading parquet fixed binary type INT96, which is used for storing hive,
+   * impala timestamp values with nanoseconds precision. So it reads such values as a drill timestamp.
+   */
+  static class NullableFixedBinaryAsTimeStampReader extends NullableFixedByteAlignedReader<NullableTimeStampVector> {
+    NullableFixedBinaryAsTimeStampReader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor,
+                              ColumnChunkMetaData columnChunkMetaData, boolean fixedLength, NullableTimeStampVector v, SchemaElement schemaElement) throws ExecutionSetupException {
+      super(parentReader, allocateSize, descriptor, columnChunkMetaData, fixedLength, v, schemaElement);
+    }
+
+    @Override
+    protected void readField(long recordsToReadInThisPass) {
+      this.bytebuf = pageReader.pageData;
+      if (usingDictionary) {
+        for (int i = 0; i < recordsToReadInThisPass; i++){
+          Binary binaryTimeStampValue = pageReader.dictionaryValueReader.readBytes();
+          valueVec.getMutator().setSafe(valuesReadInCurrentPass + i, getDateTimeValueFromBinary(binaryTimeStampValue));
+        }
+      } else {
+        for (int i = 0; i < recordsToReadInThisPass; i++) {
+          Binary binaryTimeStampValue = pageReader.valueReader.readBytes();
+          valueVec.getMutator().setSafe(valuesReadInCurrentPass + i, getDateTimeValueFromBinary(binaryTimeStampValue));
+        }
+      }
+    }
+  }
+
   static class NullableDictionaryIntReader extends NullableColumnReader<NullableIntVector> {
 
     NullableDictionaryIntReader(ParquetRecordReader parentReader, int allocateSize, ColumnDescriptor descriptor,

http://git-wip-us.apache.org/repos/asf/drill/blob/7e7214b4/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetRecordReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetRecordReader.java
index c51c72c..f095a8a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetRecordReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetRecordReader.java
@@ -248,6 +248,10 @@ public class ParquetRecordReader extends AbstractRecordReader {
     return operatorContext;
   }
 
+  public FragmentContext getFragmentContext() {
+    return fragmentContext;
+  }
+
   /**
    * Returns data type length for a given {@see ColumnDescriptor} and it's corresponding
    * {@see SchemaElement}. Neither is enough information alone as the max

http://git-wip-us.apache.org/repos/asf/drill/blob/7e7214b4/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetToDrillTypeConverter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetToDrillTypeConverter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetToDrillTypeConverter.java
index b6d1a72..57c0a66 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetToDrillTypeConverter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/ParquetToDrillTypeConverter.java
@@ -21,6 +21,7 @@ import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 
 import org.apache.drill.common.util.CoreDecimalUtility;
+import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.server.options.OptionManager;
 import org.apache.drill.exec.store.parquet.ParquetReaderUtility;
 import org.apache.parquet.format.ConvertedType;
@@ -94,7 +95,11 @@ public class ParquetToDrillTypeConverter {
       // TODO - Both of these are not supported by the parquet library yet (7/3/13),
       // but they are declared here for when they are implemented
       case INT96:
-        return TypeProtos.MinorType.VARBINARY;
+        if (options.getOption(ExecConstants.PARQUET_READER_INT96_AS_TIMESTAMP).bool_val) {
+          return TypeProtos.MinorType.TIMESTAMP;
+        } else {
+          return TypeProtos.MinorType.VARBINARY;
+        }
       case FIXED_LEN_BYTE_ARRAY:
         if (convertedType == null) {
           checkArgument(length > 0, "A length greater than zero must be provided for a FixedBinary type.");

http://git-wip-us.apache.org/repos/asf/drill/blob/7e7214b4/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet2/DrillParquetGroupConverter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet2/DrillParquetGroupConverter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet2/DrillParquetGroupConverter.java
index 48a0bfd..2f2db05 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet2/DrillParquetGroupConverter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet2/DrillParquetGroupConverter.java
@@ -28,6 +28,7 @@ import java.util.List;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.expression.PathSegment;
 import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.expr.holders.BigIntHolder;
 import org.apache.drill.exec.expr.holders.BitHolder;
 import org.apache.drill.exec.expr.holders.DateHolder;
@@ -81,6 +82,8 @@ import org.apache.parquet.schema.Type.Repetition;
 
 import com.google.common.collect.Lists;
 
+import static org.apache.drill.exec.store.parquet.ParquetReaderUtility.NanoTimeUtils.getDateTimeValueFromBinary;
+
 public class DrillParquetGroupConverter extends GroupConverter {
 
   private List<Converter> converters;
@@ -226,9 +229,15 @@ public class DrillParquetGroupConverter extends GroupConverter {
         }
       }
       case INT96: {
+        // TODO: replace null with TIMESTAMP_NANOS once parquet support such type annotation.
         if (type.getOriginalType() == null) {
-          VarBinaryWriter writer = type.getRepetition() == Repetition.REPEATED ? mapWriter.list(name).varBinary() : mapWriter.varBinary(name);
-          return new DrillFixedBinaryToVarbinaryConverter(writer, ParquetRecordReader.getTypeLengthInBits(type.getPrimitiveTypeName()) / 8, mutator.getManagedBuffer());
+          if (options.getOption(ExecConstants.PARQUET_READER_INT96_AS_TIMESTAMP).bool_val) {
+            TimeStampWriter writer = type.getRepetition() == Repetition.REPEATED ? mapWriter.list(name).timeStamp() : mapWriter.timeStamp(name);
+            return new DrillFixedBinaryToTimeStampConverter(writer);
+          } else {
+            VarBinaryWriter writer = type.getRepetition() == Repetition.REPEATED ? mapWriter.list(name).varBinary() : mapWriter.varBinary(name);
+            return new DrillFixedBinaryToVarbinaryConverter(writer, ParquetRecordReader.getTypeLengthInBits(type.getPrimitiveTypeName()) / 8, mutator.getManagedBuffer());
+          }
         }
 
       }
@@ -622,4 +631,23 @@ public class DrillParquetGroupConverter extends GroupConverter {
       writer.write(holder);
     }
   }
+
+  /**
+   * Parquet currently supports a fixed binary type INT96 for storing hive, impala timestamp
+   * with nanoseconds precision.
+   */
+  public static class DrillFixedBinaryToTimeStampConverter extends PrimitiveConverter {
+    private TimeStampWriter writer;
+    private TimeStampHolder holder = new TimeStampHolder();
+
+    public DrillFixedBinaryToTimeStampConverter(TimeStampWriter writer) {
+      this.writer = writer;
+    }
+
+    @Override
+    public void addBinary(Binary value) {
+      holder.value = getDateTimeValueFromBinary(value);
+      writer.write(holder);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/7e7214b4/exec/java-exec/src/test/java/org/apache/drill/TestBuilder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestBuilder.java b/exec/java-exec/src/test/java/org/apache/drill/TestBuilder.java
index 8acf936..a19b30e 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestBuilder.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestBuilder.java
@@ -190,12 +190,23 @@ public class TestBuilder {
     return this;
   }
 
+  public TestBuilder optionSettingQueriesForBaseline(String queries, Object... args) {
+    this.baselineOptionSettingQueries = String.format(queries, args);
+    return this;
+  }
+
   // list of queries to run before the test query, can be used to set several options
   // list takes the form of a semi-colon separated list
   public TestBuilder optionSettingQueriesForTestQuery(String queries) {
     this.testOptionSettingQueries = queries;
     return this;
   }
+
+  public TestBuilder optionSettingQueriesForTestQuery(String query, Object... args) throws Exception {
+    this.testOptionSettingQueries = String.format(query, args);
+    return this;
+  }
+
   public TestBuilder approximateEquality() {
     this.approximateEquality = true;
     return this;

http://git-wip-us.apache.org/repos/asf/drill/blob/7e7214b4/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java
index 6890394..cf43339 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java
@@ -32,7 +32,9 @@ import java.util.Map;
 
 import com.google.common.base.Joiner;
 import org.apache.drill.BaseTestQuery;
+import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.util.DrillVersionInfo;
+import org.apache.drill.common.util.TestTools;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.fn.interp.TestConstantFolding;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
@@ -739,30 +741,76 @@ public class TestParquetWriter extends BaseTestQuery {
   }
 
   /*
-  Test the reading of an int96 field. Impala encodes timestamps as int96 fields
+    Impala encodes timestamp values as int96 fields. Test the reading of an int96 field with two converters:
+    the first one converts parquet INT96 into drill VARBINARY and the second one (works while
+    store.parquet.reader.int96_as_timestamp option is enabled) converts parquet INT96 into drill TIMESTAMP.
    */
   @Test
   public void testImpalaParquetInt96() throws Exception {
     compareParquetReadersColumnar("field_impala_ts", "cp.`parquet/int96_impala_1.parquet`");
+    try {
+      test("alter session set %s = true", ExecConstants.PARQUET_READER_INT96_AS_TIMESTAMP);
+      compareParquetReadersColumnar("field_impala_ts", "cp.`parquet/int96_impala_1.parquet`");
+    } finally {
+      test("alter session reset %s", ExecConstants.PARQUET_READER_INT96_AS_TIMESTAMP);
+    }
   }
 
   /*
-  Test the reading of a binary field where data is in dicationary _and_ non-dictionary encoded pages
+  Test the reading of a binary field as drill varbinary where data is in dicationary _and_ non-dictionary encoded pages
    */
   @Test
-  public void testImpalaParquetVarBinary_DictChange() throws Exception {
+  public void testImpalaParquetBinaryAsVarBinary_DictChange() throws Exception {
     compareParquetReadersColumnar("field_impala_ts", "cp.`parquet/int96_dict_change.parquet`");
   }
 
   /*
+  Test the reading of a binary field as drill timestamp where data is in dicationary _and_ non-dictionary encoded pages
+   */
+  @Test
+  public void testImpalaParquetBinaryAsTimeStamp_DictChange() throws Exception {
+    final String WORKING_PATH = TestTools.getWorkingPath();
+    final String TEST_RES_PATH = WORKING_PATH + "/src/test/resources";
+    try {
+      testBuilder()
+          .sqlQuery("select int96_ts from dfs_test.`%s/parquet/int96_dict_change`", TEST_RES_PATH)
+          .optionSettingQueriesForTestQuery(
+              "alter session set `%s` = true", ExecConstants.PARQUET_READER_INT96_AS_TIMESTAMP)
+          .ordered()
+          .csvBaselineFile("testframework/testParquetReader/testInt96DictChange/q1.tsv")
+          .baselineTypes(TypeProtos.MinorType.TIMESTAMP)
+          .baselineColumns("int96_ts")
+          .build().run();
+    } finally {
+      test("alter system reset `%s`", ExecConstants.PARQUET_READER_INT96_AS_TIMESTAMP);
+    }
+  }
+
+  /*
      Test the conversion from int96 to impala timestamp
    */
   @Test
-  public void testImpalaParquetTimestampAsInt96() throws Exception {
+  public void testTimestampImpalaConvertFrom() throws Exception {
     compareParquetReadersColumnar("convert_from(field_impala_ts, 'TIMESTAMP_IMPALA')", "cp.`parquet/int96_impala_1.parquet`");
   }
 
   /*
+     Test reading parquet Int96 as TimeStamp and comparing obtained values with the
+     old results (reading the same values as VarBinary and convert_fromTIMESTAMP_IMPALA function using)
+   */
+  @Test
+  public void testImpalaParquetTimestampInt96AsTimeStamp() throws Exception {
+    try {
+      test("alter session set %s = false", ExecConstants.PARQUET_NEW_RECORD_READER);
+      compareParquetInt96Converters("field_impala_ts", "cp.`parquet/int96_impala_1.parquet`");
+      test("alter session set %s = true", ExecConstants.PARQUET_NEW_RECORD_READER);
+      compareParquetInt96Converters("field_impala_ts", "cp.`parquet/int96_impala_1.parquet`");
+    } finally {
+      test("alter session reset `%s`", ExecConstants.PARQUET_NEW_RECORD_READER);
+    }
+  }
+
+  /*
     Test a file with partitions and an int96 column. (Data generated using Hive)
    */
   @Test
@@ -782,7 +830,6 @@ public class TestParquetWriter extends BaseTestQuery {
   Test the conversion from int96 to impala timestamp with hive data including nulls. Validate against expected values
   */
   @Test
-  @Ignore("relies on particular time zone")
   public void testHiveParquetTimestampAsInt96_basic() throws Exception {
     final String q = "SELECT cast(convert_from(timestamp_field, 'TIMESTAMP_IMPALA') as varchar(19))  as timestamp_field "
             + "from cp.`parquet/part1/hive_all_types.parquet` ";
@@ -791,7 +838,7 @@ public class TestParquetWriter extends BaseTestQuery {
             .unOrdered()
             .sqlQuery(q)
             .baselineColumns("timestamp_field")
-            .baselineValues("2013-07-05 17:01:00")
+            .baselineValues("2013-07-06 00:01:00")
             .baselineValues((Object)null)
             .go();
   }
@@ -859,5 +906,22 @@ public class TestParquetWriter extends BaseTestQuery {
         "cp.`parquet/last_page_one_null.parquet`");
   }
 
+  private void compareParquetInt96Converters(String selection, String table) throws Exception {
+    try {
+      testBuilder()
+          .ordered()
+          .sqlQuery("select `%s` from %s", selection, table)
+          .optionSettingQueriesForTestQuery(
+              "alter session set `%s` = true", ExecConstants.PARQUET_READER_INT96_AS_TIMESTAMP)
+          .sqlBaselineQuery("select convert_from(`%1$s`, 'TIMESTAMP_IMPALA') as `%1$s` from %2$s", selection, table)
+          .optionSettingQueriesForBaseline(
+              "alter session set `%s` = false", ExecConstants.PARQUET_READER_INT96_AS_TIMESTAMP)
+          .build()
+          .run();
+    } finally {
+      test("alter system reset `%s`", ExecConstants.PARQUET_READER_INT96_AS_TIMESTAMP);
+    }
+  }
+
 }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/7e7214b4/exec/java-exec/src/test/resources/parquet/int96_dict_change/000000_0
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/parquet/int96_dict_change/000000_0 b/exec/java-exec/src/test/resources/parquet/int96_dict_change/000000_0
new file mode 100644
index 0000000..8517428
Binary files /dev/null and b/exec/java-exec/src/test/resources/parquet/int96_dict_change/000000_0 differ

http://git-wip-us.apache.org/repos/asf/drill/blob/7e7214b4/exec/java-exec/src/test/resources/parquet/int96_dict_change/000000_1
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/parquet/int96_dict_change/000000_1 b/exec/java-exec/src/test/resources/parquet/int96_dict_change/000000_1
new file mode 100644
index 0000000..0183b50
Binary files /dev/null and b/exec/java-exec/src/test/resources/parquet/int96_dict_change/000000_1 differ

http://git-wip-us.apache.org/repos/asf/drill/blob/7e7214b4/exec/java-exec/src/test/resources/testframework/testParquetReader/testInt96DictChange/q1.tsv
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/testframework/testParquetReader/testInt96DictChange/q1.tsv b/exec/java-exec/src/test/resources/testframework/testParquetReader/testInt96DictChange/q1.tsv
new file mode 100644
index 0000000..91b9b01
--- /dev/null
+++ b/exec/java-exec/src/test/resources/testframework/testParquetReader/testInt96DictChange/q1.tsv
@@ -0,0 +1,12 @@
+1970-01-01 00:00:01.000
+1971-01-01 00:00:01.000
+1972-01-01 00:00:01.000
+1973-01-01 00:00:01.000
+1974-01-01 00:00:01.000
+2010-01-01 00:00:01.000
+2011-01-01 00:00:01.000
+2012-01-01 00:00:01.000
+2013-01-01 00:00:01.000
+2014-01-01 00:00:01.000
+2015-01-01 00:00:01.000
+2016-01-01 00:00:01.000


[12/15] drill git commit: DRILL-4420: C++ API for metadata access and prepared statements

Posted by pa...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/166c4ce7/contrib/native/client/src/clientlib/drillClientImpl.hpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/clientlib/drillClientImpl.hpp b/contrib/native/client/src/clientlib/drillClientImpl.hpp
index 06f37e0..f9d0779 100644
--- a/contrib/native/client/src/clientlib/drillClientImpl.hpp
+++ b/contrib/native/client/src/clientlib/drillClientImpl.hpp
@@ -21,7 +21,6 @@
 #define DRILL_CLIENT_IMPL_H
 
 #include "drill/common.hpp"
-
 // Define some BOOST defines
 // WIN32_SHUTDOWN_ON_TIMEOUT is defined in "drill/common.hpp" for Windows 32 bit platform
 #ifndef WIN32_SHUTDOWN_ON_TIMEOUT
@@ -29,27 +28,25 @@
 #endif //WIN32_SHUTDOWN_ON_TIMEOUT
 
 #include <algorithm>
-#include <stdlib.h>
-#include <time.h>
 #include <queue>
 #include <vector>
-#include <boost/asio.hpp>
 
+#include <boost/asio.hpp>
 #if defined _WIN32  || defined _WIN64
-#include <zookeeper.h>
 //Windows header files redefine 'random'
 #ifdef random
 #undef random
 #endif
-#else
-#include <zookeeper/zookeeper.h>
 #endif
 #include <boost/asio/deadline_timer.hpp>
+#include <boost/function.hpp>
 #include <boost/thread.hpp>
 
 #include "drill/drillClient.hpp"
-#include "rpcEncoder.hpp"
-#include "rpcDecoder.hpp"
+#include "drill/preparedStatement.hpp"
+#include "collectionsImpl.hpp"
+#include "metadata.hpp"
+#include "rpcMessage.hpp"
 #include "utils.hpp"
 #include "User.pb.h"
 #include "UserBitShared.pb.h"
@@ -57,11 +54,11 @@
 namespace Drill {
 
 class DrillClientImpl;
-class InBoundRpcMessage;
-class OutBoundRpcMessage;
+
+class DrillClientQueryHandle;
+
+class DrillClientPrepareHandle;
 class RecordBatch;
-class RpcEncoder;
-class RpcDecoder;
 
 /*
  * Defines the interface used by DrillClient and implemented by DrillClientImpl and PooledDrillClientImpl
@@ -89,6 +86,8 @@ class DrillClientImplBase{
 
         // Submits a query to a drillbit. 
         virtual DrillClientQueryResult* SubmitQuery(::exec::shared::QueryType t, const std::string& plan, pfnQueryResultsListener listener, void* listenerCtx)=0;
+        virtual DrillClientPrepareHandle* PrepareQuery(const std::string& plan, pfnPreparedStatementListener listener, void* listenerCtx)=0;
+        virtual DrillClientQueryResult* ExecuteQuery(const PreparedStatement& pstmt, pfnQueryResultsListener listener, void* listenerCtx)=0;
 
         //Waits as a connection has results pending
         virtual void waitForResults()=0;
@@ -96,31 +95,109 @@ class DrillClientImplBase{
         //Validates handshake at connect time.
         virtual connectionStatus_t validateHandshake(DrillUserProperties* props)=0;
 
-        virtual void freeQueryResources(DrillClientQueryResult* pQryResult)=0;
+        virtual void freeQueryResources(DrillClientQueryHandle* pQryHandle)=0;
+
+        virtual meta::DrillMetadata* getMetadata() = 0;
 
+        virtual void freeMetadata(meta::DrillMetadata* metadata) = 0;
 };
 
-class DrillClientQueryResult{
+/**
+ * Base type for query handles
+ */
+class DrillClientQueryHandle{
     friend class DrillClientImpl;
     public:
-    DrillClientQueryResult(DrillClientImpl * pClient, uint64_t coordId, const std::string& query):
-        m_pClient(pClient),
+    DrillClientQueryHandle(DrillClientImpl& client, int32_t coordId, const std::string& query, void* context):
+        m_client(client),
         m_coordinationId(coordId),
         m_query(query),
+		m_status(QRY_SUCCESS),
+        m_bCancel(false),
+        m_bHasError(false),
+        m_pError(NULL),
+		m_pApplicationContext(context){
+    };
+
+    virtual ~DrillClientQueryHandle(){
+        clearAndDestroy();
+    };
+
+    virtual void cancel();
+    bool isCancelled() const {return m_bCancel;};
+    int32_t getCoordinationId() const { return m_coordinationId;}
+    const std::string&  getQuery() const { return m_query;}
+
+    bool hasError() const { return m_bHasError;}
+    void resetError() { m_bHasError = false; }
+
+    status_t getErrorStatus() const { return m_pError!=NULL?(status_t)m_pError->status:QRY_SUCCESS;}
+    const DrillClientError* getError() const { return m_pError;}
+    void setQueryStatus(status_t s){ m_status = s;}
+    status_t getQueryStatus() const { return m_status;}
+    inline DrillClientImpl& client() const { return m_client; };
+
+    inline void* getApplicationContext() const { return m_pApplicationContext; }
+
+    protected:
+
+    virtual void signalError(DrillClientError* pErr);
+    virtual void clearAndDestroy();
+
+    private:
+    DrillClientImpl& m_client;
+
+    int32_t m_coordinationId;
+    std::string m_query;
+    status_t m_status;
+    bool m_bCancel;
+    bool m_bHasError;
+
+    const DrillClientError* m_pError;
+
+    void* m_pApplicationContext;
+};
+
+template<typename Listener, typename ListenerValue>
+class DrillClientBaseHandle: public DrillClientQueryHandle {
+    friend class DrillClientImpl;
+    public:
+    DrillClientBaseHandle(DrillClientImpl& client, int32_t coordId, const std::string& query, Listener listener, void* context):
+    	DrillClientQueryHandle(client, coordId, query, context),
+		m_pApplicationListener(listener){
+    };
+
+    virtual ~DrillClientBaseHandle(){
+        clearAndDestroy();
+    };
+
+    inline Listener getApplicationListener() const { return m_pApplicationListener; }
+
+
+    protected:
+    virtual status_t notifyListener(ListenerValue v, DrillClientError* pErr);
+
+    virtual void signalError(DrillClientError* pErr);
+    void setHasError(bool hasError) { m_bHasError = hasError; }
+
+    private:
+    Listener m_pApplicationListener;
+};
+
+class DrillClientQueryResult: public DrillClientBaseHandle<pfnQueryResultsListener, RecordBatch*>{
+    friend class DrillClientImpl;
+    public:
+    DrillClientQueryResult(DrillClientImpl& client, int32_t coordId, const std::string& query, pfnQueryResultsListener listener, void* listenerCtx):
+    	DrillClientBaseHandle<pfnQueryResultsListener, RecordBatch*>(client, coordId, query, listener, listenerCtx),
         m_numBatches(0),
         m_columnDefs(new std::vector<Drill::FieldMetadata*>),
         m_bIsQueryPending(true),
         m_bIsLastChunk(false),
-        m_bCancel(false),
         m_bHasSchemaChanged(false),
         m_bHasData(false),
-        m_bHasError(false),
         m_queryState(exec::shared::QueryResult_QueryState_STARTING),
-        m_pError(NULL),
         m_pQueryId(NULL),
-        m_pSchemaListener(NULL),
-        m_pResultsListener(NULL),
-        m_pListenerCtx(NULL) {
+        m_pSchemaListener(NULL) {
     };
 
     ~DrillClientQueryResult(){
@@ -128,20 +205,15 @@ class DrillClientQueryResult{
     };
 
     // get data asynchronously
-    void registerListener(pfnQueryResultsListener listener, void* listenerCtx){
-        this->m_pResultsListener=listener;
-        this->m_pListenerCtx = listenerCtx;
-    }
-
     void registerSchemaChangeListener(pfnSchemaListener l){
         m_pSchemaListener=l;
     }
 
-    // Synchronous call to get data. Caller assumes ownership of the recod batch
+    // Synchronous call to get data. Caller assumes ownership of the record batch
     // returned and it is assumed to have been consumed.
     RecordBatch*  getNext();
     // Synchronous call to get a look at the next Record Batch. This
-    // call does not move the current pointer forward. Repeatied calls
+    // call does not move the current pointer forward. Repeated calls
     // to peekNext return the same value until getNext is called.
     RecordBatch*  peekNext();
     // Blocks until data is available.
@@ -150,32 +222,26 @@ class DrillClientQueryResult{
     // placeholder to return an empty col def vector when calls are made out of order.
     static FieldDefPtr s_emptyColDefs;
 
-    FieldDefPtr getColumnDefs(){
+    FieldDefPtr getColumnDefs() {
         boost::lock_guard<boost::mutex> bufferLock(this->m_schemaMutex);
         return this->m_columnDefs;
     }
 
-    void cancel();
-    bool isCancelled(){return this->m_bCancel;};
-    bool hasSchemaChanged(){return this->m_bHasSchemaChanged;};
-    int32_t getCoordinationId(){ return this->m_coordinationId;}
-    const std::string&  getQuery(){ return this->m_query;}
+    bool hasSchemaChanged() const {return this->m_bHasSchemaChanged;};
 
     void setQueryId(exec::shared::QueryId* q){this->m_pQueryId=q;}
-    void* getListenerContext() {return this->m_pListenerCtx;}
-    exec::shared::QueryId& getQueryId(){ return *(this->m_pQueryId); }
-    bool hasError(){ return m_bHasError;}
-    status_t getErrorStatus(){ return m_pError!=NULL?(status_t)m_pError->status:QRY_SUCCESS;}
-    const DrillClientError* getError(){ return m_pError;}
-    void setQueryStatus(status_t s){ m_status = s;}
-    status_t getQueryStatus(){ return m_status;}
+    exec::shared::QueryId& getQueryId() const { return *(this->m_pQueryId); }
 
     void setQueryState(exec::shared::QueryResult_QueryState s){ m_queryState = s;}
-    exec::shared::QueryResult_QueryState getQueryState(){ return m_queryState;}
+    exec::shared::QueryResult_QueryState getQueryState() const { return m_queryState;}
     void setIsQueryPending(bool isPending){
         boost::lock_guard<boost::mutex> cvLock(this->m_cvMutex);
         m_bIsQueryPending=isPending;
     }
+    protected:
+    virtual status_t notifyListener(RecordBatch* batch, DrillClientError* pErr);
+    virtual void signalError(DrillClientError* pErr);
+    virtual void clearAndDestroy();
 
     private:
     status_t setupColumnDefs(exec::shared::QueryData* pQueryData);
@@ -183,15 +249,7 @@ class DrillClientQueryResult{
     // Construct a DrillClientError object, set the appropriate state and signal any listeners, condition variables.
     // Also used when a query is cancelled or when a query completed response is received.
     // Error object is now owned by the DrillClientQueryResult object.
-    void signalError(DrillClientError* pErr);
     void signalComplete();
-    void clearAndDestroy();
-
-
-    DrillClientImpl* m_pClient;
-
-    int32_t m_coordinationId;
-    const std::string& m_query;
 
     size_t m_numBatches; // number of record batches received so far
 
@@ -213,28 +271,90 @@ class DrillClientQueryResult{
     // if m_bIsQueryPending is true, we continue to wait for results
     bool m_bIsQueryPending;
     bool m_bIsLastChunk;
-    bool m_bCancel;
     bool m_bHasSchemaChanged;
     bool m_bHasData;
-    bool m_bHasError;
 
     // state in the last query result received from the server.
     exec::shared::QueryResult_QueryState m_queryState;
 
-    const DrillClientError* m_pError;
-
     exec::shared::QueryId* m_pQueryId;
-    status_t m_status;
 
     // Schema change listener
     pfnSchemaListener m_pSchemaListener;
-    // Results callback
-    pfnQueryResultsListener m_pResultsListener;
+};
+
+class DrillClientPrepareHandle: public DrillClientBaseHandle<pfnPreparedStatementListener, PreparedStatement*>, public PreparedStatement {
+    public:
+    DrillClientPrepareHandle(DrillClientImpl& client, int32_t coordId, const std::string& query, pfnPreparedStatementListener listener, void* listenerCtx):
+    	DrillClientBaseHandle<pfnPreparedStatementListener, PreparedStatement*>(client, coordId, query, listener, listenerCtx),
+		PreparedStatement(),
+        m_columnDefs(new std::vector<Drill::FieldMetadata*>) {
+    };
+
+    // PreparedStatement overrides
+	virtual std::size_t getNumFields() const { return m_columnDefs->size(); }
+	virtual const Drill::FieldMetadata& getFieldMetadata(std::size_t index) const { return *m_columnDefs->at(index);}
+
+    protected:
+    virtual void clearAndDestroy();
+
+    private:
+    friend class DrillClientImpl;
+    status_t setupPreparedStatement(const exec::user::PreparedStatement& pstmt);
+
+    FieldDefPtr m_columnDefs;
+    ::exec::user::PreparedStatementHandle m_preparedStatementHandle;
+};
+
+template<typename Listener, typename MetaType, typename MetaImpl, typename MetadataResult>
+class DrillClientMetadataResult: public DrillClientBaseHandle<Listener, const DrillCollection<MetaType>*> {
+public:
+    DrillClientMetadataResult(DrillClientImpl& client, int32_t coordId, const std::string& query, Listener listener, void* listenerCtx):
+    	DrillClientBaseHandle<Listener, const DrillCollection<MetaType>*>(client, coordId, query, listener, listenerCtx) {}
+
+    void attachMetadataResult(MetadataResult* result) { this->m_pMetadata.reset(result); }
+
+private:
+    friend class DrillClientImpl;
+
+    // Meta informations returned to the user, linked to the handle
+    DrillVector<MetaType, MetaImpl> m_meta;
+
+    // to keep a reference on the underlying metadata object, and
+    // make sure it's clean when this handle is destroyed
+    boost::shared_ptr<MetadataResult> m_pMetadata;
+
+};
+
+class DrillClientCatalogResult: public DrillClientMetadataResult<Metadata::pfnCatalogMetadataListener, meta::CatalogMetadata, meta::DrillCatalogMetadata, exec::user::GetCatalogsResp> {
+    friend class DrillClientImpl;
+public:
+    DrillClientCatalogResult(DrillClientImpl& client, int32_t coordId, Metadata::pfnCatalogMetadataListener listener, void* listenerCtx):
+    	DrillClientMetadataResult<Metadata::pfnCatalogMetadataListener, meta::CatalogMetadata, meta::DrillCatalogMetadata, exec::user::GetCatalogsResp>(client, coordId, "getCatalog", listener, listenerCtx) {}
+};
+
+class DrillClientSchemaResult: public DrillClientMetadataResult<Metadata::pfnSchemaMetadataListener, meta::SchemaMetadata, meta::DrillSchemaMetadata, exec::user::GetSchemasResp> {
+    friend class DrillClientImpl;
+public:
+    DrillClientSchemaResult(DrillClientImpl& client, int32_t coordId, Metadata::pfnSchemaMetadataListener listener, void* listenerCtx):
+    	DrillClientMetadataResult<Metadata::pfnSchemaMetadataListener, meta::SchemaMetadata, meta::DrillSchemaMetadata, exec::user::GetSchemasResp>(client, coordId, "getSchemas", listener, listenerCtx) {}
+};
+
+class DrillClientTableResult: public DrillClientMetadataResult<Metadata::pfnTableMetadataListener, meta::TableMetadata, meta::DrillTableMetadata, exec::user::GetTablesResp> {
+    friend class DrillClientImpl;
+public:
+    DrillClientTableResult(DrillClientImpl& client, int32_t coordId, Metadata::pfnTableMetadataListener listener, void* listenerCtx):
+    	DrillClientMetadataResult<Metadata::pfnTableMetadataListener, meta::TableMetadata, meta::DrillTableMetadata, exec::user::GetTablesResp>(client, coordId, "getTables", listener, listenerCtx) {}
+};
 
-    // Listener context
-    void * m_pListenerCtx;
+class DrillClientColumnResult: public DrillClientMetadataResult<Metadata::pfnColumnMetadataListener, meta::ColumnMetadata, meta::DrillColumnMetadata, exec::user::GetColumnsResp> {
+    friend class DrillClientImpl;
+    public:
+    DrillClientColumnResult(DrillClientImpl& client, int32_t coordId, Metadata::pfnColumnMetadataListener listener, void* listenerCtx):
+    	DrillClientMetadataResult<Metadata::pfnColumnMetadataListener, meta::ColumnMetadata, meta::DrillColumnMetadata, exec::user::GetColumnsResp>(client, coordId, "getColumns", listener, listenerCtx) {}
 };
 
+
 class DrillClientImpl : public DrillClientImplBase{
     public:
         DrillClientImpl():
@@ -250,7 +370,8 @@ class DrillClientImpl : public DrillClientImplBase{
             m_deadlineTimer(m_io_service),
             m_heartbeatTimer(m_io_service),
             m_rbuf(NULL),
-            m_wbuf(MAX_SOCK_RD_BUFSIZE)
+            m_wbuf(MAX_SOCK_RD_BUFSIZE),
+			m_bIsDirectConnection(false)
     {
         m_coordinationId=rand()%1729+1;
     };
@@ -300,14 +421,24 @@ class DrillClientImpl : public DrillClientImplBase{
         void Close() ;
         DrillClientError* getError(){ return m_pError;}
         DrillClientQueryResult* SubmitQuery(::exec::shared::QueryType t, const std::string& plan, pfnQueryResultsListener listener, void* listenerCtx);
+        DrillClientPrepareHandle* PrepareQuery(const std::string& plan, pfnPreparedStatementListener listener, void* listenerCtx);
+        DrillClientQueryResult* ExecuteQuery(const PreparedStatement& pstmt, pfnQueryResultsListener listener, void* listenerCtx);
+
         void waitForResults();
         connectionStatus_t validateHandshake(DrillUserProperties* props);
-        void freeQueryResources(DrillClientQueryResult* pQryResult){
-            // Doesn't need to do anything
-            return;
+        void freeQueryResources(DrillClientQueryHandle* pQryHandle){
+            delete pQryHandle;
         };
+        
+        const exec::user::RpcEndpointInfos& getServerInfos() const { return m_serverInfos; }
+
+        meta::DrillMetadata* getMetadata();
+
+        void freeMetadata(meta::DrillMetadata* metadata);
 
     private:
+        friend class meta::DrillMetadata;
+        friend class DrillClientQueryHandle;
         friend class DrillClientQueryResult;
         friend class PooledDrillClientImpl;
 
@@ -327,8 +458,8 @@ class DrillClientImpl : public DrillClientImplBase{
 
         int32_t getNextCoordinationId(){ return ++m_coordinationId; };
         // send synchronous messages
-        //connectionStatus_t recvSync(InBoundRpcMessage& msg);
-        connectionStatus_t sendSync(OutBoundRpcMessage& msg);
+        //connectionStatus_t recvSync(rpc::InBoundRpcMessage& msg);
+        connectionStatus_t sendSync(rpc::OutBoundRpcMessage& msg);
         // handshake
         connectionStatus_t recvHandshake();
         void handleHandshake(ByteBuf_t b, const boost::system::error_code& err, std::size_t bytes_transferred );
@@ -340,45 +471,54 @@ class DrillClientImpl : public DrillClientImplBase{
         status_t readMsg(
                 ByteBuf_t _buf,
                 AllocatedBufferPtr* allocatedBuffer,
-                InBoundRpcMessage& msg,
-                boost::system::error_code& error);
-        status_t processQueryResult(AllocatedBufferPtr allocatedBuffer, InBoundRpcMessage& msg);
-        status_t processQueryData(AllocatedBufferPtr allocatedBuffer, InBoundRpcMessage& msg);
+                rpc::InBoundRpcMessage& msg);
+        status_t processQueryResult(AllocatedBufferPtr allocatedBuffer, const rpc::InBoundRpcMessage& msg);
+        status_t processQueryData(AllocatedBufferPtr allocatedBuffer, const rpc::InBoundRpcMessage& msg);
         status_t processCancelledQueryResult( exec::shared::QueryId& qid, exec::shared::QueryResult* qr);
-        status_t processQueryId(AllocatedBufferPtr allocatedBuffer, InBoundRpcMessage& msg );
-        DrillClientQueryResult* findQueryResult(exec::shared::QueryId& qid);
+        status_t processQueryId(AllocatedBufferPtr allocatedBuffer, const rpc::InBoundRpcMessage& msg );
+        status_t processPreparedStatement(AllocatedBufferPtr allocatedBuffer, const rpc::InBoundRpcMessage& msg );
+        status_t processCatalogsResult(AllocatedBufferPtr allocatedBuffer, const rpc::InBoundRpcMessage& msg );
+        status_t processSchemasResult(AllocatedBufferPtr allocatedBuffer, const rpc::InBoundRpcMessage& msg );
+        status_t processTablesResult(AllocatedBufferPtr allocatedBuffer, const rpc::InBoundRpcMessage& msg );
+        status_t processColumnsResult(AllocatedBufferPtr allocatedBuffer, const rpc::InBoundRpcMessage& msg );
+        DrillClientQueryResult* findQueryResult(const exec::shared::QueryId& qid);
         status_t processQueryStatusResult( exec::shared::QueryResult* qr,
                 DrillClientQueryResult* pDrillClientQueryResult);
         void handleReadTimeout(const boost::system::error_code & err);
         void handleRead(ByteBuf_t _buf, const boost::system::error_code & err, size_t bytes_transferred) ;
-        status_t validateDataMessage(InBoundRpcMessage& msg, exec::shared::QueryData& qd, std::string& valError);
-        status_t validateResultMessage(InBoundRpcMessage& msg, exec::shared::QueryResult& qr, std::string& valError);
-        connectionStatus_t handleConnError(connectionStatus_t status, std::string msg);
-        status_t handleQryError(status_t status, std::string msg, DrillClientQueryResult* pQueryResult);
-        status_t handleQryError(status_t status,
-                const exec::shared::DrillPBError& e,
-                DrillClientQueryResult* pQueryResult);
-        // handle query state indicating query is COMPELTED or CANCELED
-        // (i.e., COMPELTED or CANCELED)
+        status_t validateDataMessage(const rpc::InBoundRpcMessage& msg, const exec::shared::QueryData& qd, std::string& valError);
+        status_t validateResultMessage(const rpc::InBoundRpcMessage& msg, const exec::shared::QueryResult& qr, std::string& valError);
+        connectionStatus_t handleConnError(connectionStatus_t status, const std::string& msg);
+        status_t handleQryError(status_t status, const std::string& msg, DrillClientQueryHandle* pQueryHandle);
+        status_t handleQryError(status_t status, const exec::shared::DrillPBError& e, DrillClientQueryHandle* pQueryHandle);
+        // handle query state indicating query is COMPLETED or CANCELED
+        // (i.e., COMPLETED or CANCELED)
         status_t handleTerminatedQryState(status_t status,
-                std::string msg,
+                const std::string& msg,
                 DrillClientQueryResult* pQueryResult);
         void broadcastError(DrillClientError* pErr);
-        void clearMapEntries(DrillClientQueryResult* pQueryResult);
-        void sendAck(InBoundRpcMessage& msg, bool isOk);
-        void sendCancel(exec::shared::QueryId* pQueryId);
+        void removeQueryHandle(DrillClientQueryHandle* pQueryHandle);
+        void removeQueryResult(DrillClientQueryResult* pQueryResult);
+        void sendAck(const rpc::InBoundRpcMessage& msg, bool isOk);
+        void sendCancel(const exec::shared::QueryId* pQueryId);
 
-        void shutdownSocket();
+        template<typename Handle>
+        Handle* sendMsg(boost::function<Handle*(int32_t)> handleFactory, ::exec::user::RpcType type, const ::google::protobuf::Message& msg);
 
+        // metadata requests
+        DrillClientCatalogResult* getCatalogs(const std::string& catalogPattern, Metadata::pfnCatalogMetadataListener listener, void* listenerCtx);
+        DrillClientSchemaResult* getSchemas(const std::string& catalogPattern, const std::string& schemaPattern, Metadata::pfnSchemaMetadataListener listener, void* listenerCtx);
+        DrillClientTableResult* getTables(const std::string& catalogPattern, const std::string& schemaPattern, const std::string& tablePattern, const std::vector<std::string>* tableTypes, Metadata::pfnTableMetadataListener listener, void* listenerCtx);
+        DrillClientColumnResult* getColumns(const std::string& catalogPattern, const std::string& schemaPattern, const std::string& tablePattern, const std::string& columnPattern, Metadata::pfnColumnMetadataListener listener, void* listenerCtx);
 
-        static RpcEncoder s_encoder;
-        static RpcDecoder s_decoder;
+        void shutdownSocket();
 
         int32_t m_coordinationId;
         int32_t m_handshakeVersion;
         exec::user::HandshakeStatus m_handshakeStatus;
         std::string m_handshakeErrorId;
         std::string m_handshakeErrorMsg;
+        exec::user::RpcEndpointInfos m_serverInfos;
         bool m_bIsConnected;
 
         std::string m_connectStr; 
@@ -418,8 +558,8 @@ class DrillClientImpl : public DrillClientImplBase{
         // Mutex to protect drill client operations
         boost::mutex m_dcMutex;
 
-        // Map of coordination id to  Query Ids.
-        std::map<int, DrillClientQueryResult*> m_queryIds;
+        // Map of coordination id to Query handles.
+        std::map<int, DrillClientQueryHandle*> m_queryHandles;
 
         // Map of query id to query result for currently executing queries
         std::map<exec::shared::QueryId*, DrillClientQueryResult*, compareQueryId> m_queryResults;
@@ -431,7 +571,7 @@ class DrillClientImpl : public DrillClientImplBase{
 };
 
 inline bool DrillClientImpl::Active() {
-    return this->m_bIsConnected;;
+    return this->m_bIsConnected;
 }
 
 
@@ -442,17 +582,17 @@ inline bool DrillClientImpl::Active() {
  * */
 class PooledDrillClientImpl : public DrillClientImplBase{
     public:
-        PooledDrillClientImpl(){
-            m_bIsDirectConnection=false;
-            m_maxConcurrentConnections = DEFAULT_MAX_CONCURRENT_CONNECTIONS;
+        PooledDrillClientImpl():
+        	m_lastConnection(-1),
+			m_queriesExecuted(0),
+			m_maxConcurrentConnections(DEFAULT_MAX_CONCURRENT_CONNECTIONS),
+			m_bIsDirectConnection(false),
+			m_pError(NULL),
+			m_pUserProperties() {
             char* maxConn=std::getenv(MAX_CONCURRENT_CONNECTIONS_ENV);
             if(maxConn!=NULL){
                 m_maxConcurrentConnections=atoi(maxConn);
             }
-            m_lastConnection=-1;
-            m_pError=NULL;
-            m_queriesExecuted=0;
-            m_pUserProperties=NULL;
         }
 
         ~PooledDrillClientImpl(){
@@ -460,7 +600,6 @@ class PooledDrillClientImpl : public DrillClientImplBase{
                 delete *it;
             }
             m_clientConnections.clear();
-            if(m_pUserProperties!=NULL){ delete m_pUserProperties; m_pUserProperties=NULL;}
             if(m_pError!=NULL){ delete m_pError; m_pError=NULL;}
         }
 
@@ -482,15 +621,22 @@ class PooledDrillClientImpl : public DrillClientImplBase{
         // Connections once added to the pool will be removed only when the DrillClient is closed.
         DrillClientQueryResult* SubmitQuery(::exec::shared::QueryType t, const std::string& plan, pfnQueryResultsListener listener, void* listenerCtx);
 
+        DrillClientPrepareHandle* PrepareQuery(const std::string& plan, pfnPreparedStatementListener listener, void* listenerCtx);
+        DrillClientQueryResult* ExecuteQuery(const PreparedStatement& pstmt, pfnQueryResultsListener listener, void* listenerCtx);
+
         //Waits as long as any one drillbit connection has results pending
         void waitForResults();
 
         //Validates handshake only against the first drillbit connected to.
         connectionStatus_t validateHandshake(DrillUserProperties* props);
 
-        void freeQueryResources(DrillClientQueryResult* pQryResult);
+        void freeQueryResources(DrillClientQueryHandle* pQueryHandle);
+
+        int getDrillbitCount() const { return m_drillbits.size();};
+        
+        meta::DrillMetadata* getMetadata();
 
-        int getDrillbitCount(){ return m_drillbits.size();};
+        void freeMetadata(meta::DrillMetadata* metadata);
 
     private:
         
@@ -502,9 +648,6 @@ class PooledDrillClientImpl : public DrillClientImplBase{
         // is currently executing. If none,  
         std::vector<DrillClientImpl*> m_clientConnections; 
 		boost::mutex m_poolMutex; // protect access to the vector
-        
-        //ZookeeperImpl zook;
-        
         // Use this to decide which drillbit to select next from the list of drillbits.
         size_t m_lastConnection;
 		boost::mutex m_cMutex;
@@ -524,44 +667,7 @@ class PooledDrillClientImpl : public DrillClientImplBase{
 
         std::vector<std::string> m_drillbits;
 
-        DrillUserProperties* m_pUserProperties;//Keep a copy of user properties
-};
-
-class ZookeeperImpl{
-    public:
-        ZookeeperImpl();
-        ~ZookeeperImpl();
-        static ZooLogLevel getZkLogLevel();
-        // comma separated host:port pairs, each corresponding to a zk
-        // server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002
-        DEPRECATED int connectToZookeeper(const char* connectStr, const char* pathToDrill);
-        void close();
-        static void watcher(zhandle_t *zzh, int type, int state, const char *path, void* context);
-        void debugPrint();
-        std::string& getError(){return m_err;}
-        const exec::DrillbitEndpoint& getEndPoint(){ return m_drillServiceInstance.endpoint();}
-        // return unshuffled list of drillbits
-        int getAllDrillbits(const char* connectStr, const char* pathToDrill, std::vector<std::string>& drillbits);
-        // picks the index drillbit and returns the corresponding endpoint object
-        int getEndPoint(std::vector<std::string>& drillbits, size_t index, exec::DrillbitEndpoint& endpoint);
-        
-
-    private:
-        static char s_drillRoot[];
-        static char s_defaultCluster[];
-        zhandle_t* m_zh;
-        clientid_t m_id;
-        int m_state;
-        std::string m_err;
-
-        struct String_vector* m_pDrillbits;
-
-        boost::mutex m_cvMutex;
-        // Condition variable to signal connection callback has been processed
-        boost::condition_variable m_cv;
-        bool m_bConnecting;
-        exec::DrillServiceInstance m_drillServiceInstance;
-        std::string m_rootDir;
+        boost::shared_ptr<DrillUserProperties> m_pUserProperties;//Keep a copy of user properties
 };
 
 } // namespace Drill

http://git-wip-us.apache.org/repos/asf/drill/blob/166c4ce7/contrib/native/client/src/clientlib/env.h.in
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/clientlib/env.h.in b/contrib/native/client/src/clientlib/env.h.in
index a32f152..746a500 100644
--- a/contrib/native/client/src/clientlib/env.h.in
+++ b/contrib/native/client/src/clientlib/env.h.in
@@ -19,6 +19,15 @@
 #ifndef ENV_H
 #define ENV_H
 
+#define DRILL_NAME              "Apache Drill"
+#define DRILL_CONNECTOR_NAME    "Apache Drill C++ client"
+#define DRILL_VERSION_STRING    "@PROJECT_VERSION@"
+
+#define DRILL_VERSION_MAJOR @PROJECT_VERSION_MAJOR@
+#define DRILL_VERSION_MINOR @PROJECT_VERSION_MINOR@
+#define DRILL_VERSION_PATCH @PROJECT_VERSION_PATCH@
+
+#define GIT_SHA_PROP  @GIT_SHA_PROP@
 #define GIT_COMMIT_PROP @GIT_COMMIT_PROP@
 
 #endif

http://git-wip-us.apache.org/repos/asf/drill/blob/166c4ce7/contrib/native/client/src/clientlib/errmsgs.cpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/clientlib/errmsgs.cpp b/contrib/native/client/src/clientlib/errmsgs.cpp
index 47d165f..56510ec 100644
--- a/contrib/native/client/src/clientlib/errmsgs.cpp
+++ b/contrib/native/client/src/clientlib/errmsgs.cpp
@@ -74,7 +74,7 @@ static Drill::ErrorMessages errorMessages[]={
 std::string getMessage(uint32_t msgId, ...){
     char str[10240];
     std::string s;
-    assert(msgId <= ERR_QRY_MAX);
+    assert((ERR_NONE <= msgId) && (msgId < ERR_QRY_MAX));
     va_list args;
     va_start (args, msgId);
     vsprintf (str, errorMessages[msgId-DRILL_ERR_START].msgFormatStr, args);

http://git-wip-us.apache.org/repos/asf/drill/blob/166c4ce7/contrib/native/client/src/clientlib/fieldmeta.cpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/clientlib/fieldmeta.cpp b/contrib/native/client/src/clientlib/fieldmeta.cpp
new file mode 100644
index 0000000..d9d6bd1
--- /dev/null
+++ b/contrib/native/client/src/clientlib/fieldmeta.cpp
@@ -0,0 +1,406 @@
+/*
+ * 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.
+ */
+
+#include "drill/common.hpp"
+#include "drill/fieldmeta.hpp"
+#include "../protobuf/UserBitShared.pb.h"
+#include "../protobuf/User.pb.h"
+
+namespace {
+// List of SQL types as string constants
+static std::string SQLAny("ANY");
+static std::string SQLArray("ARRAY");
+static std::string SQLBigint("BIGINT");
+static std::string SQLBinary("BINARY");
+static std::string SQLBoolean("BOOLEAN");
+static std::string SQLChar("CHARACTER");
+static std::string SQLDate("DATE");
+static std::string SQLDecimal("DECIMAL");
+static std::string SQLDouble("DOUBLE");
+static std::string SQLFloat("FLOAT");
+static std::string SQLInteger("INTEGER");
+static std::string SQLInterval("INTERVAL");
+static std::string SQLIntervalYearMonth("INTERVAL YEAR TO MONTH");
+static std::string SQLIntervalDaySecond("INTERVAL DAY TO SECOND");
+static std::string SQLNChar("NATIONAL CHARACTER");
+static std::string SQLNull("NULL");
+static std::string SQLMap("MAP");
+static std::string SQLSmallint("SMALLINT");
+static std::string SQLTime("TIME");
+static std::string SQLTimestamp("TIMESTAMP");
+static std::string SQLTimestampTZ("TIMESTAMP WITH TIME ZONE");
+static std::string SQLTimeTZ("TIME WITH TIME ZONE");
+static std::string SQLTinyint("TINYINT");
+static std::string SQLUnion("UNION");
+static std::string SQLVarbinary("BINARY VARYING");
+static std::string SQLVarchar("CHARACTER VARYING");
+static std::string SQLVarnchar("NATIONAL CHARACTER VARYING");
+static std::string SQLUnknown("__unknown__");
+
+static const std::string& getSQLType(common::MinorType type, common::DataMode mode) {
+  if (mode == common::DM_REPEATED || type == common::LIST) {
+    return SQLArray;
+  }
+
+  switch(type) {
+    case common::BIT:             return SQLBoolean;
+
+    case common::TINYINT:         return SQLTinyint;
+    case common::SMALLINT:        return SQLSmallint;
+    case common::INT:             return SQLInteger;
+    case common::BIGINT:          return SQLBigint;
+    case common::FLOAT4:          return SQLFloat;
+    case common::FLOAT8:          return SQLDouble;
+
+    case common::DECIMAL9:
+    case common::DECIMAL18:
+    case common::DECIMAL28DENSE:
+    case common::DECIMAL28SPARSE:
+    case common::DECIMAL38DENSE:
+    case common::DECIMAL38SPARSE: return SQLDecimal;
+
+    case common::VARCHAR:         return SQLVarchar;
+    case common::FIXEDCHAR:       return SQLChar;
+
+    case common::VAR16CHAR:       return SQLVarnchar;
+    case common::FIXED16CHAR:     return SQLNChar;
+
+    case common::VARBINARY:       return SQLVarbinary;
+    case common::FIXEDBINARY:     return SQLBinary;
+
+    case common::DATE:            return SQLDate;
+    case common::TIME:            return SQLTime;
+    case common::TIMETZ:          return SQLTimeTZ;
+    case common::TIMESTAMP:       return SQLTimestamp;
+    case common::TIMESTAMPTZ:     return SQLTimestampTZ;
+
+    case common::INTERVALYEAR:    return SQLIntervalYearMonth;
+    case common::INTERVALDAY:     return SQLIntervalDaySecond;
+    case common::INTERVAL:        return SQLInterval;
+    case common::MONEY:           return SQLDecimal;
+
+    case common::MAP:             return SQLMap;
+    case common::LATE:            return SQLAny;
+    case common::DM_UNKNOWN:      return SQLNull;
+    case common::UNION:           return SQLUnion;
+
+    case common::UINT1:           return SQLTinyint;
+    case common::UINT2:           return SQLSmallint;
+    case common::UINT4:           return SQLInteger;
+    case common::UINT8:           return SQLBigint;
+
+    default:
+      return SQLUnknown;
+  }
+}
+
+static bool isSortable(common::MinorType type) {
+  return type != common::MAP && type != common::LIST;
+}
+
+static bool isNullable(common::DataMode mode) {
+  return mode == common::DM_OPTIONAL; // Same behaviour as JDBC
+}
+
+static bool isSigned(common::MinorType type, common::DataMode mode) {
+  if (mode == common::DM_REPEATED) {
+    return false;// SQL ARRAY
+  }
+
+  switch(type) {
+    case common::SMALLINT:
+    case common::INT:
+    case common::BIGINT:
+    case common::FLOAT4:
+    case common::FLOAT8:
+
+    case common::DECIMAL9:
+    case common::DECIMAL18:
+    case common::DECIMAL28DENSE:
+    case common::DECIMAL38DENSE:
+    case common::DECIMAL38SPARSE:
+
+    case common::INTERVALYEAR:
+    case common::INTERVALDAY:
+    case common::INTERVAL:
+    case common::MONEY:
+    case common::TINYINT:
+      return true;
+
+    case common::BIT:
+    case common::VARCHAR:
+    case common::FIXEDCHAR:
+
+    case common::VAR16CHAR:
+    case common::FIXED16CHAR:
+
+    case common::VARBINARY:
+    case common::FIXEDBINARY:
+
+    case common::DATE:
+    case common::TIME:
+    case common::TIMETZ:
+    case common::TIMESTAMP:
+    case common::TIMESTAMPTZ:
+
+    case common::MAP:
+    case common::LATE:
+    case common::DM_UNKNOWN:
+    case common::UNION:
+
+    case common::UINT1:
+    case common::UINT2:
+    case common::UINT4:
+    case common::UINT8:
+      return false;
+
+    default:
+      return false;
+  }
+}
+
+static Drill::FieldMetadata::ColumnSearchability getSearchability(exec::user::ColumnSearchability s) {
+  switch(s) {
+    case exec::user::UNKNOWN_SEARCHABILITY: return Drill::FieldMetadata::UNKNOWN_SEARCHABILITY;
+    case exec::user::NONE:                  return Drill::FieldMetadata::NONE;
+    case exec::user::CHAR:                  return Drill::FieldMetadata::CHAR;
+    case exec::user::NUMBER:                return Drill::FieldMetadata::NUMBER;
+    case exec::user::ALL:                   return Drill::FieldMetadata::ALL;
+
+    default:
+      return Drill::FieldMetadata::UNKNOWN_SEARCHABILITY;
+  }
+}
+
+static Drill::FieldMetadata::ColumnUpdatability getUpdatability(exec::user::ColumnUpdatability u) {
+  switch(u) {
+    case exec::user::UNKNOWN_UPDATABILITY: return Drill::FieldMetadata::UNKNOWN_UPDATABILITY;
+    case exec::user::READ_ONLY:            return Drill::FieldMetadata::READ_ONLY;
+    case exec::user::WRITABLE:             return Drill::FieldMetadata::WRITABLE;
+
+    default:
+      return Drill::FieldMetadata::UNKNOWN_UPDATABILITY;
+  }
+}
+
+// Based on ODBC spec
+// https://msdn.microsoft.com/en-us/library/ms711786(v=vs.85).aspx
+static uint32_t getColumnSize(const std::string& type, uint32_t precision) {
+	if (type == SQLBoolean) {
+		return 1;
+	}
+	else if (type == SQLTinyint) {
+		return 3;
+	}
+	else if (type == SQLSmallint) {
+		return 5;
+	}
+	else if (type == SQLInteger) {
+		return 10;
+	}
+	else if (type == SQLBigint) {
+		return 19;
+	}
+	else if (type == SQLFloat) {
+		return 7;
+	}
+	else if (type == SQLDouble) {
+		return 15;
+	}
+	else if (type == SQLDecimal) {
+		return precision;
+	}
+	else if (type == SQLBinary || type == SQLVarbinary
+			|| type == SQLChar || type == SQLVarchar
+			|| type == SQLNChar || type == SQLVarnchar) {
+		return precision;
+	}
+	else if (type == SQLDate) {
+		return 10; // 'yyyy-MM-dd' format
+	}
+	else if (type == SQLTime) {
+		if (precision > 0) {
+			return 9 + precision;
+		}
+		else return 8; // 'hh-mm-ss' format
+	}
+	else if (type == SQLTimestamp) {
+		return (precision > 0)
+			? 20 + precision
+			: 19; // 'yyyy-MM-ddThh-mm-ss' format
+	}
+	else if (type == SQLIntervalYearMonth) {
+		return (precision > 0)
+				? 5 + precision // P..M31
+				: 0; // if precision is not set, return 0 because there's not enough info
+	}
+	else if (type == SQLIntervalDaySecond) {
+		return (precision > 0)
+			? 12 + precision // P..DT12H60M60....S
+			: 0; // if precision is not set, return 0 because there's not enough info
+	}
+	else {
+		return 0;
+	}
+}
+
+static uint32_t getPrecision(const ::common::MajorType& type) {
+	const ::common::MinorType& minor_type = type.minor_type();
+
+	if (type.has_precision()) {
+		return type.precision();
+	}
+
+	if (minor_type == ::common::VARBINARY || minor_type == ::common::VARCHAR) {
+		return 65536;
+	}
+
+	return 0;
+}
+
+// From Types.java
+// Based on ODBC spec:
+// https://msdn.microsoft.com/en-us/library/ms713974(v=vs.85).aspx
+static uint32_t getDisplaySize(const ::common::MajorType& type) {
+    if (type.mode() == ::common::DM_REPEATED || type.minor_type() == ::common::LIST) {
+      return 0;
+    }
+
+    uint32_t precision = getPrecision(type);
+
+    switch(type.minor_type()) {
+    case ::common::BIT:             return 1; // 1 digit
+
+    case ::common::TINYINT:         return 4; // sign + 3 digit
+    case ::common::SMALLINT:        return 6; // sign + 5 digits
+    case ::common::INT:             return 11; // sign + 10 digits
+    case ::common::BIGINT:          return 20; // sign + 19 digits
+
+    case ::common::UINT1:          return 3; // 3 digits
+    case ::common::UINT2:          return 5; // 5 digits
+    case ::common::UINT4:          return 10; // 10 digits
+    case ::common::UINT8:          return 19; // 19 digits
+
+    case ::common::FLOAT4:          return 14; // sign + 7 digits + decimal point + E + 2 digits
+    case ::common::FLOAT8:          return 24; // sign + 15 digits + decimal point + E + 3 digits
+
+    case ::common::DECIMAL9:
+    case ::common::DECIMAL18:
+    case ::common::DECIMAL28DENSE:
+    case ::common::DECIMAL28SPARSE:
+    case ::common::DECIMAL38DENSE:
+    case ::common::DECIMAL38SPARSE:
+    case ::common::MONEY:           return 2 + precision; // precision of the column plus a sign and a decimal point
+
+    case ::common::VARCHAR:
+    case ::common::FIXEDCHAR:
+    case ::common::VAR16CHAR:
+    case ::common::FIXED16CHAR:     return precision; // number of characters
+
+    case ::common::VARBINARY:
+    case ::common::FIXEDBINARY:     return 2 * precision; // each binary byte is represented as a 2digit hex number
+
+    case ::common::DATE:            return 10; // yyyy-mm-dd
+    case ::common::TIME:
+      return precision > 0
+        ? 9 + precision // hh-mm-ss.SSS
+        : 8; // hh-mm-ss
+    case ::common::TIMETZ:
+      return precision > 0
+        ? 15 + precision // hh-mm-ss.SSS-zz:zz
+        : 14; // hh-mm-ss-zz:zz
+    case ::common::TIMESTAMP:
+      return precision > 0
+         ? 20 + precision // yyyy-mm-ddThh:mm:ss.SSS
+         : 19; // yyyy-mm-ddThh:mm:ss
+    case ::common::TIMESTAMPTZ:
+      return precision > 0
+        ? 26 + precision // yyyy-mm-ddThh:mm:ss.SSS:ZZ-ZZ
+        : 25; // yyyy-mm-ddThh:mm:ss-ZZ:ZZ
+
+    case ::common::INTERVALYEAR:
+      return precision > 0
+          ? 5 + precision // P..Y12M
+          : 0; // if precision is not set, return 0 because there's not enough info
+
+    case ::common::INTERVALDAY:
+      return precision > 0
+          ? 12 + precision // P..DT12H60M60S assuming fractional seconds precision is not supported
+          : 0; // if precision is not set, return 0 because there's not enough info
+
+    default:
+    	// We don't know how to compute a display size, let's return 0 (unknown)
+    	return 0;
+}
+}
+} // namespace
+
+namespace Drill{
+
+void FieldMetadata::set(const exec::shared::SerializedField& f){
+    m_name=f.name_part().name();
+    m_minorType=f.major_type().minor_type();
+    m_dataMode=f.major_type().mode();
+    m_valueCount=f.value_count();
+    m_scale=f.major_type().scale();
+    m_precision=f.major_type().precision();
+    m_bufferLength=f.buffer_length();
+    m_catalogName="DRILL";
+    m_schemaName=""; // unknown
+    m_tableName=""; // unknown;
+    m_label=m_name;
+    m_sqlType=::getSQLType(m_minorType, m_dataMode);
+    m_nullable=::isNullable(m_dataMode);
+    m_signed=::isSigned(m_minorType, m_dataMode);
+    m_displaySize=::getDisplaySize(f.major_type());
+    m_searchability=ALL;
+    m_updatability=READ_ONLY;
+    m_autoIncremented=false;
+    m_caseSensitive=false;
+    m_sortable=::isSortable(m_minorType);
+    m_currency=false;
+    m_columnSize = ::getColumnSize(m_sqlType, m_precision);
+}
+
+void FieldMetadata::set(const exec::user::ResultColumnMetadata& m){
+    m_name=m.column_name();
+    m_minorType=static_cast<common::MinorType>(-1);
+    m_dataMode=static_cast<common::DataMode>(-1);
+    m_valueCount=0;
+    m_scale=m.scale();
+    m_precision=m.precision();
+    m_bufferLength=0;
+    m_catalogName=m.catalog_name();
+    m_schemaName=m.schema_name();
+    m_tableName=m.table_name();
+    m_label=m.label();
+    m_sqlType=m.data_type();
+    m_nullable=m.is_nullable();
+    m_displaySize=m.display_size();
+    m_signed=m.signed_();
+    m_searchability=::getSearchability(m.searchability());
+    m_updatability=::getUpdatability(m.updatability());
+    m_autoIncremented=m.auto_increment();
+    m_caseSensitive=m.case_sensitivity();
+    m_sortable=m.sortable();
+    m_currency=m.is_currency();
+    m_columnSize =::getColumnSize(m_sqlType, m_precision);
+}
+
+}// namespace Drill
+

http://git-wip-us.apache.org/repos/asf/drill/blob/166c4ce7/contrib/native/client/src/clientlib/metadata.cpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/clientlib/metadata.cpp b/contrib/native/client/src/clientlib/metadata.cpp
new file mode 100644
index 0000000..0364c7d
--- /dev/null
+++ b/contrib/native/client/src/clientlib/metadata.cpp
@@ -0,0 +1,748 @@
+/*
+ * 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.
+ */
+
+#include <boost/assign.hpp>
+#include <boost/functional/hash.hpp>
+#include <boost/unordered_set.hpp>
+#include "drillClientImpl.hpp"
+
+#include "metadata.hpp"
+
+const std::string Drill::meta::DrillMetadata::s_connectorName(DRILL_CONNECTOR_NAME);
+const std::string Drill::meta::DrillMetadata::s_connectorVersion(DRILL_VERSION_STRING);
+
+const std::string Drill::meta::DrillMetadata::s_catalogSeparator(".");
+const std::string Drill::meta::DrillMetadata::s_catalogTerm("catalog");
+const std::string Drill::meta::DrillMetadata::s_identifierQuoteString("`");
+
+const std::vector<std::string> Drill::meta::DrillMetadata::s_sqlKeywords = boost::assign::list_of
+		("ABS")("ALLOW")("ARRAY")("ASENSITIVE")("ASYMMETRIC")("ATOMIC")("BIGINT")("BINARY")("BLOB")
+		("BOOLEAN")("CALL")("CALLED")("CARDINALITY")("CEIL")("CEILING")("CLOB")("COLLECT")("CONDITION")
+		("CORR")("COVAR_POP")("COVAR_SAMP")("CUBE")("CUME_DIST")("CURRENT_CATALOG")
+		("CURRENT_DEFAULT_TRANSFORM_GROUP")("CURRENT_PATH")("CURRENT_ROLE")("CURRENT_SCHEMA")
+		("CURRENT_TRANSFORM_GROUP_FOR_TYPE")("CYCLE")("DATABASE")("DATABASES")("DENSE_RANK")("DEREF")
+		("DETERMINISTIC")("DISALLOW")("DYNAMIC")("EACH")("ELEMENT")("EVERY")("EXP")("EXPLAIN")
+		("EXTEND")("FILES")("FILTER")("FIRST_VALUE")("FLOOR")("FREE")("FUNCTION")("FUSION")("GROUPING")
+		("HOLD")("IF")("IMPORT")("INOUT")("INTERSECTION")("LARGE")("LAST_VALUE")("LATERAL")("LIMIT")("LN")
+		("LOCALTIME")("LOCALTIMESTAMP")("MEMBER")("MERGE")("METADATA")("METHOD")("MOD")("MODIFIES")
+		("MULTISET")("NCLOB")("NEW")("NONE")("NORMALIZE")("OFFSET")("OLD")("OUT")("OVER")("OVERLAY")
+		("PARAMETER")("PARTITION")("PERCENTILE_CONT")("PERCENTILE_DISC")("PERCENT_RANK")("POWER")
+		("RANGE")("RANK")("READS")("RECURSIVE")("REF")("REFERENCING")("REFRESH")("REGR_AVGX")("REGR_AVGY")
+		("REGR_COUNT")("REGR_INTERCEPT")("REGR_R2")("REGR_SLOPE")("REGR_SXX")("REGR_SXY")("REGR_SYY")
+		("RELEASE")("REPLACE")("RESET")("RESULT")("RETURN")("RETURNS")("ROLLUP")("ROW")("ROW_NUMBER")
+		("SAVEPOINT")("SCHEMAS")("SCOPE")("SEARCH")("SENSITIVE")("SHOW")("SIMILAR")("SPECIFIC")("SPECIFICTYPE")
+		("SQLEXCEPTION")("SQLWARNING")("SQRT")("START")("STATIC")("STDDEV_POP")("STDDEV_SAMP")("STREAM")
+		("SUBMULTISET")("SYMMETRIC")("SYSTEM")("TABLES")("TABLESAMPLE")("TINYINT")("TREAT")("TRIGGER")
+		("UESCAPE")("UNNEST")("UPSERT")("USE")("VARBINARY")("VAR_POP")("VAR_SAMP")("WIDTH_BUCKET")
+		("WINDOW")("WITHIN")("WITHOUT");
+
+const std::vector<std::string> Drill::meta::DrillMetadata::s_numericFunctions = boost::assign::list_of
+		("ABS")("ACOS")("ASIN")("ATAN")("ATAN2")("CEILING")("COS")("COT")
+		("DEGREES")("EXP")("FLOOR")("LOG")("LOG10")("MOD")("PI")
+		("POWER")("RADIANS")("RAND")("ROUND")("SIGN")("SIN")("SQRT")
+		("TAN")("TRUNCATE");
+
+const std::string Drill::meta::DrillMetadata::s_schemaTerm("schema");
+const std::string Drill::meta::DrillMetadata::s_searchEscapeString("\\");
+const std::string Drill::meta::DrillMetadata::s_specialCharacters;
+
+const std::vector<std::string> Drill::meta::DrillMetadata::s_stringFunctions = boost::assign::list_of
+		("ASCII")("CHAR")("CONCAT")("DIFFERENCE")("INSERT")("LCASE")
+		("LEFT")("LENGTH")("LOCATE")("LTRIM")("REPEAT")("REPLACE")
+		("RIGHT")("RTRIM")("SOUNDEX")("SPACE")("SUBSTRING")("UCASE");
+
+const std::vector<std::string> Drill::meta::DrillMetadata::s_systemFunctions = boost::assign::list_of
+		("DATABASE")("IFNULL")("USER");
+
+const std::string Drill::meta::DrillMetadata::s_tableTerm("table");
+
+const std::vector<std::string> Drill::meta::DrillMetadata::s_dateTimeFunctions = boost::assign::list_of
+		("CURDATE")("CURTIME")("DAYNAME")("DAYOFMONTH")("DAYOFWEEK")
+		("DAYOFYEAR")("HOUR")("MINUTE")("MONTH")("MONTHNAME")("NOW")
+		("QUARTER")("SECOND")("TIMESTAMPADD")("TIMESTAMPDIFF")("WEEK")("YEAR");
+
+namespace Drill {
+namespace meta {
+namespace {
+using boost::assign::list_of;
+
+struct FromTo {
+	FromTo(common::MinorType from, common::MinorType to): m_from(from), m_to(to) {}
+
+	common::MinorType m_from;
+	common::MinorType m_to;
+};
+
+bool operator==(FromTo const& ft1, FromTo const& ft2) {
+	return ft1.m_from == ft2.m_from && ft1.m_to == ft2.m_to;
+}
+
+std::size_t hash_value(FromTo const& ft) {
+	std::size_t hash = 0;
+	boost::hash_combine(hash, ft.m_from);
+	boost::hash_combine(hash, ft.m_to);
+
+	return hash;
+}
+
+static boost::unordered_set<FromTo> s_convertMap = boost::assign::list_of
+		(FromTo(common::TINYINT, common::INT))
+		(FromTo(common::TINYINT, common::BIGINT))
+		(FromTo(common::TINYINT, common::DECIMAL9))
+		(FromTo(common::TINYINT, common::DECIMAL18))
+		(FromTo(common::TINYINT, common::DECIMAL28SPARSE))
+		(FromTo(common::TINYINT, common::DECIMAL38SPARSE))
+		(FromTo(common::TINYINT, common::DATE))
+		(FromTo(common::TINYINT, common::TIME))
+		(FromTo(common::TINYINT, common::TIMESTAMP))
+		(FromTo(common::TINYINT, common::INTERVAL))
+		(FromTo(common::TINYINT, common::FLOAT4))
+		(FromTo(common::TINYINT, common::FLOAT8))
+		(FromTo(common::TINYINT, common::BIT))
+		(FromTo(common::TINYINT, common::VARCHAR))
+		(FromTo(common::TINYINT, common::VAR16CHAR))
+		(FromTo(common::TINYINT, common::VARBINARY))
+		(FromTo(common::TINYINT, common::INTERVALYEAR))
+		(FromTo(common::TINYINT, common::INTERVALDAY))
+		(FromTo(common::SMALLINT, common::INT))
+		(FromTo(common::SMALLINT, common::BIGINT))
+		(FromTo(common::SMALLINT, common::DECIMAL9))
+		(FromTo(common::SMALLINT, common::DECIMAL18))
+		(FromTo(common::SMALLINT, common::DECIMAL28SPARSE))
+		(FromTo(common::SMALLINT, common::DECIMAL38SPARSE))
+		(FromTo(common::SMALLINT, common::DATE))
+		(FromTo(common::SMALLINT, common::TIME))
+		(FromTo(common::SMALLINT, common::TIMESTAMP))
+		(FromTo(common::SMALLINT, common::INTERVAL))
+		(FromTo(common::SMALLINT, common::FLOAT4))
+		(FromTo(common::SMALLINT, common::FLOAT8))
+		(FromTo(common::SMALLINT, common::BIT))
+		(FromTo(common::SMALLINT, common::VARCHAR))
+		(FromTo(common::SMALLINT, common::VAR16CHAR))
+		(FromTo(common::SMALLINT, common::VARBINARY))
+		(FromTo(common::SMALLINT, common::INTERVALYEAR))
+		(FromTo(common::SMALLINT, common::INTERVALDAY))
+		(FromTo(common::INT, common::INT))
+		(FromTo(common::INT, common::BIGINT))
+		(FromTo(common::INT, common::DECIMAL9))
+		(FromTo(common::INT, common::DECIMAL18))
+		(FromTo(common::INT, common::DECIMAL28SPARSE))
+		(FromTo(common::INT, common::DECIMAL38SPARSE))
+		(FromTo(common::INT, common::DATE))
+		(FromTo(common::INT, common::TIME))
+		(FromTo(common::INT, common::TIMESTAMP))
+		(FromTo(common::INT, common::INTERVAL))
+		(FromTo(common::INT, common::FLOAT4))
+		(FromTo(common::INT, common::FLOAT8))
+		(FromTo(common::INT, common::BIT))
+		(FromTo(common::INT, common::VARCHAR))
+		(FromTo(common::INT, common::VAR16CHAR))
+		(FromTo(common::INT, common::VARBINARY))
+		(FromTo(common::INT, common::INTERVALYEAR))
+		(FromTo(common::INT, common::INTERVALDAY))
+		(FromTo(common::BIGINT, common::INT))
+		(FromTo(common::BIGINT, common::BIGINT))
+		(FromTo(common::BIGINT, common::DECIMAL9))
+		(FromTo(common::BIGINT, common::DECIMAL18))
+		(FromTo(common::BIGINT, common::DECIMAL28SPARSE))
+		(FromTo(common::BIGINT, common::DECIMAL38SPARSE))
+		(FromTo(common::BIGINT, common::DATE))
+		(FromTo(common::BIGINT, common::TIME))
+		(FromTo(common::BIGINT, common::TIMESTAMP))
+		(FromTo(common::BIGINT, common::INTERVAL))
+		(FromTo(common::BIGINT, common::FLOAT4))
+		(FromTo(common::BIGINT, common::FLOAT8))
+		(FromTo(common::BIGINT, common::BIT))
+		(FromTo(common::BIGINT, common::VARCHAR))
+		(FromTo(common::BIGINT, common::VAR16CHAR))
+		(FromTo(common::BIGINT, common::VARBINARY))
+		(FromTo(common::BIGINT, common::INTERVALYEAR))
+		(FromTo(common::BIGINT, common::INTERVALDAY))
+		(FromTo(common::DECIMAL9, common::INT))
+		(FromTo(common::DECIMAL9, common::BIGINT))
+		(FromTo(common::DECIMAL9, common::DECIMAL9))
+		(FromTo(common::DECIMAL9, common::DECIMAL18))
+		(FromTo(common::DECIMAL9, common::DECIMAL28SPARSE))
+		(FromTo(common::DECIMAL9, common::DECIMAL38SPARSE))
+		(FromTo(common::DECIMAL9, common::DATE))
+		(FromTo(common::DECIMAL9, common::TIME))
+		(FromTo(common::DECIMAL9, common::TIMESTAMP))
+		(FromTo(common::DECIMAL9, common::INTERVAL))
+		(FromTo(common::DECIMAL9, common::FLOAT4))
+		(FromTo(common::DECIMAL9, common::FLOAT8))
+		(FromTo(common::DECIMAL9, common::BIT))
+		(FromTo(common::DECIMAL9, common::VARCHAR))
+		(FromTo(common::DECIMAL9, common::VAR16CHAR))
+		(FromTo(common::DECIMAL9, common::VARBINARY))
+		(FromTo(common::DECIMAL9, common::INTERVALYEAR))
+		(FromTo(common::DECIMAL9, common::INTERVALDAY))
+		(FromTo(common::DECIMAL18, common::INT))
+		(FromTo(common::DECIMAL18, common::BIGINT))
+		(FromTo(common::DECIMAL18, common::DECIMAL9))
+		(FromTo(common::DECIMAL18, common::DECIMAL18))
+		(FromTo(common::DECIMAL18, common::DECIMAL28SPARSE))
+		(FromTo(common::DECIMAL18, common::DECIMAL38SPARSE))
+		(FromTo(common::DECIMAL18, common::DATE))
+		(FromTo(common::DECIMAL18, common::TIME))
+		(FromTo(common::DECIMAL18, common::TIMESTAMP))
+		(FromTo(common::DECIMAL18, common::INTERVAL))
+		(FromTo(common::DECIMAL18, common::FLOAT4))
+		(FromTo(common::DECIMAL18, common::FLOAT8))
+		(FromTo(common::DECIMAL18, common::BIT))
+		(FromTo(common::DECIMAL18, common::VARCHAR))
+		(FromTo(common::DECIMAL18, common::VAR16CHAR))
+		(FromTo(common::DECIMAL18, common::VARBINARY))
+		(FromTo(common::DECIMAL18, common::INTERVALYEAR))
+		(FromTo(common::DECIMAL18, common::INTERVALDAY))
+		(FromTo(common::DECIMAL28SPARSE, common::INT))
+		(FromTo(common::DECIMAL28SPARSE, common::BIGINT))
+		(FromTo(common::DECIMAL28SPARSE, common::DECIMAL9))
+		(FromTo(common::DECIMAL28SPARSE, common::DECIMAL18))
+		(FromTo(common::DECIMAL28SPARSE, common::DECIMAL28SPARSE))
+		(FromTo(common::DECIMAL28SPARSE, common::DECIMAL38SPARSE))
+		(FromTo(common::DECIMAL28SPARSE, common::DATE))
+		(FromTo(common::DECIMAL28SPARSE, common::TIME))
+		(FromTo(common::DECIMAL28SPARSE, common::TIMESTAMP))
+		(FromTo(common::DECIMAL28SPARSE, common::INTERVAL))
+		(FromTo(common::DECIMAL28SPARSE, common::FLOAT4))
+		(FromTo(common::DECIMAL28SPARSE, common::FLOAT8))
+		(FromTo(common::DECIMAL28SPARSE, common::BIT))
+		(FromTo(common::DECIMAL28SPARSE, common::VARCHAR))
+		(FromTo(common::DECIMAL28SPARSE, common::VAR16CHAR))
+		(FromTo(common::DECIMAL28SPARSE, common::VARBINARY))
+		(FromTo(common::DECIMAL28SPARSE, common::INTERVALYEAR))
+		(FromTo(common::DECIMAL28SPARSE, common::INTERVALDAY))
+		(FromTo(common::DECIMAL38SPARSE, common::INT))
+		(FromTo(common::DECIMAL38SPARSE, common::BIGINT))
+		(FromTo(common::DECIMAL38SPARSE, common::DECIMAL9))
+		(FromTo(common::DECIMAL38SPARSE, common::DECIMAL18))
+		(FromTo(common::DECIMAL38SPARSE, common::DECIMAL28SPARSE))
+		(FromTo(common::DECIMAL38SPARSE, common::DECIMAL38SPARSE))
+		(FromTo(common::DECIMAL38SPARSE, common::DATE))
+		(FromTo(common::DECIMAL38SPARSE, common::TIME))
+		(FromTo(common::DECIMAL38SPARSE, common::TIMESTAMP))
+		(FromTo(common::DECIMAL38SPARSE, common::INTERVAL))
+		(FromTo(common::DECIMAL38SPARSE, common::FLOAT4))
+		(FromTo(common::DECIMAL38SPARSE, common::FLOAT8))
+		(FromTo(common::DECIMAL38SPARSE, common::BIT))
+		(FromTo(common::DECIMAL38SPARSE, common::VARCHAR))
+		(FromTo(common::DECIMAL38SPARSE, common::VAR16CHAR))
+		(FromTo(common::DECIMAL38SPARSE, common::VARBINARY))
+		(FromTo(common::DECIMAL38SPARSE, common::INTERVALYEAR))
+		(FromTo(common::DECIMAL38SPARSE, common::INTERVALDAY))
+		(FromTo(common::MONEY, common::INT))
+		(FromTo(common::MONEY, common::BIGINT))
+		(FromTo(common::MONEY, common::DECIMAL9))
+		(FromTo(common::MONEY, common::DECIMAL18))
+		(FromTo(common::MONEY, common::DECIMAL28SPARSE))
+		(FromTo(common::MONEY, common::DECIMAL38SPARSE))
+		(FromTo(common::MONEY, common::DATE))
+		(FromTo(common::MONEY, common::TIME))
+		(FromTo(common::MONEY, common::TIMESTAMP))
+		(FromTo(common::MONEY, common::INTERVAL))
+		(FromTo(common::MONEY, common::FLOAT4))
+		(FromTo(common::MONEY, common::FLOAT8))
+		(FromTo(common::MONEY, common::BIT))
+		(FromTo(common::MONEY, common::VARCHAR))
+		(FromTo(common::MONEY, common::VAR16CHAR))
+		(FromTo(common::MONEY, common::VARBINARY))
+		(FromTo(common::MONEY, common::INTERVALYEAR))
+		(FromTo(common::MONEY, common::INTERVALDAY))
+		(FromTo(common::DATE, common::INT))
+		(FromTo(common::DATE, common::BIGINT))
+		(FromTo(common::DATE, common::DECIMAL9))
+		(FromTo(common::DATE, common::DECIMAL18))
+		(FromTo(common::DATE, common::DECIMAL28SPARSE))
+		(FromTo(common::DATE, common::DECIMAL38SPARSE))
+		(FromTo(common::DATE, common::DATE))
+		(FromTo(common::DATE, common::TIME))
+		(FromTo(common::DATE, common::TIMESTAMP))
+		(FromTo(common::DATE, common::INTERVAL))
+		(FromTo(common::DATE, common::FLOAT4))
+		(FromTo(common::DATE, common::FLOAT8))
+		(FromTo(common::DATE, common::BIT))
+		(FromTo(common::DATE, common::VARCHAR))
+		(FromTo(common::DATE, common::VAR16CHAR))
+		(FromTo(common::DATE, common::VARBINARY))
+		(FromTo(common::DATE, common::INTERVALYEAR))
+		(FromTo(common::DATE, common::INTERVALDAY))
+		(FromTo(common::TIME, common::INT))
+		(FromTo(common::TIME, common::BIGINT))
+		(FromTo(common::TIME, common::DECIMAL9))
+		(FromTo(common::TIME, common::DECIMAL18))
+		(FromTo(common::TIME, common::DECIMAL28SPARSE))
+		(FromTo(common::TIME, common::DECIMAL38SPARSE))
+		(FromTo(common::TIME, common::DATE))
+		(FromTo(common::TIME, common::TIME))
+		(FromTo(common::TIME, common::TIMESTAMP))
+		(FromTo(common::TIME, common::INTERVAL))
+		(FromTo(common::TIME, common::FLOAT4))
+		(FromTo(common::TIME, common::FLOAT8))
+		(FromTo(common::TIME, common::BIT))
+		(FromTo(common::TIME, common::VARCHAR))
+		(FromTo(common::TIME, common::VAR16CHAR))
+		(FromTo(common::TIME, common::VARBINARY))
+		(FromTo(common::TIME, common::INTERVALYEAR))
+		(FromTo(common::TIME, common::INTERVALDAY))
+		(FromTo(common::TIMESTAMPTZ, common::INT))
+		(FromTo(common::TIMESTAMPTZ, common::BIGINT))
+		(FromTo(common::TIMESTAMPTZ, common::DECIMAL9))
+		(FromTo(common::TIMESTAMPTZ, common::DECIMAL18))
+		(FromTo(common::TIMESTAMPTZ, common::DECIMAL28SPARSE))
+		(FromTo(common::TIMESTAMPTZ, common::DECIMAL38SPARSE))
+		(FromTo(common::TIMESTAMPTZ, common::DATE))
+		(FromTo(common::TIMESTAMPTZ, common::TIME))
+		(FromTo(common::TIMESTAMPTZ, common::TIMESTAMP))
+		(FromTo(common::TIMESTAMPTZ, common::INTERVAL))
+		(FromTo(common::TIMESTAMPTZ, common::FLOAT4))
+		(FromTo(common::TIMESTAMPTZ, common::FLOAT8))
+		(FromTo(common::TIMESTAMPTZ, common::BIT))
+		(FromTo(common::TIMESTAMPTZ, common::VARCHAR))
+		(FromTo(common::TIMESTAMPTZ, common::VAR16CHAR))
+		(FromTo(common::TIMESTAMPTZ, common::VARBINARY))
+		(FromTo(common::TIMESTAMPTZ, common::INTERVALYEAR))
+		(FromTo(common::TIMESTAMPTZ, common::INTERVALDAY))
+		(FromTo(common::TIMESTAMP, common::INT))
+		(FromTo(common::TIMESTAMP, common::BIGINT))
+		(FromTo(common::TIMESTAMP, common::DECIMAL9))
+		(FromTo(common::TIMESTAMP, common::DECIMAL18))
+		(FromTo(common::TIMESTAMP, common::DECIMAL28SPARSE))
+		(FromTo(common::TIMESTAMP, common::DECIMAL38SPARSE))
+		(FromTo(common::TIMESTAMP, common::DATE))
+		(FromTo(common::TIMESTAMP, common::TIME))
+		(FromTo(common::TIMESTAMP, common::TIMESTAMP))
+		(FromTo(common::TIMESTAMP, common::INTERVAL))
+		(FromTo(common::TIMESTAMP, common::FLOAT4))
+		(FromTo(common::TIMESTAMP, common::FLOAT8))
+		(FromTo(common::TIMESTAMP, common::BIT))
+		(FromTo(common::TIMESTAMP, common::VARCHAR))
+		(FromTo(common::TIMESTAMP, common::VAR16CHAR))
+		(FromTo(common::TIMESTAMP, common::VARBINARY))
+		(FromTo(common::TIMESTAMP, common::INTERVALYEAR))
+		(FromTo(common::TIMESTAMP, common::INTERVALDAY))
+		(FromTo(common::INTERVAL, common::INT))
+		(FromTo(common::INTERVAL, common::BIGINT))
+		(FromTo(common::INTERVAL, common::DECIMAL9))
+		(FromTo(common::INTERVAL, common::DECIMAL18))
+		(FromTo(common::INTERVAL, common::DECIMAL28SPARSE))
+		(FromTo(common::INTERVAL, common::DECIMAL38SPARSE))
+		(FromTo(common::INTERVAL, common::DATE))
+		(FromTo(common::INTERVAL, common::TIME))
+		(FromTo(common::INTERVAL, common::TIMESTAMP))
+		(FromTo(common::INTERVAL, common::INTERVAL))
+		(FromTo(common::INTERVAL, common::FLOAT4))
+		(FromTo(common::INTERVAL, common::FLOAT8))
+		(FromTo(common::INTERVAL, common::BIT))
+		(FromTo(common::INTERVAL, common::VARCHAR))
+		(FromTo(common::INTERVAL, common::VAR16CHAR))
+		(FromTo(common::INTERVAL, common::VARBINARY))
+		(FromTo(common::INTERVAL, common::INTERVALYEAR))
+		(FromTo(common::INTERVAL, common::INTERVALDAY))
+		(FromTo(common::FLOAT4, common::INT))
+		(FromTo(common::FLOAT4, common::BIGINT))
+		(FromTo(common::FLOAT4, common::DECIMAL9))
+		(FromTo(common::FLOAT4, common::DECIMAL18))
+		(FromTo(common::FLOAT4, common::DECIMAL28SPARSE))
+		(FromTo(common::FLOAT4, common::DECIMAL38SPARSE))
+		(FromTo(common::FLOAT4, common::DATE))
+		(FromTo(common::FLOAT4, common::TIME))
+		(FromTo(common::FLOAT4, common::TIMESTAMP))
+		(FromTo(common::FLOAT4, common::INTERVAL))
+		(FromTo(common::FLOAT4, common::FLOAT4))
+		(FromTo(common::FLOAT4, common::FLOAT8))
+		(FromTo(common::FLOAT4, common::BIT))
+		(FromTo(common::FLOAT4, common::VARCHAR))
+		(FromTo(common::FLOAT4, common::VAR16CHAR))
+		(FromTo(common::FLOAT4, common::VARBINARY))
+		(FromTo(common::FLOAT4, common::INTERVALYEAR))
+		(FromTo(common::FLOAT4, common::INTERVALDAY))
+		(FromTo(common::FLOAT8, common::INT))
+		(FromTo(common::FLOAT8, common::BIGINT))
+		(FromTo(common::FLOAT8, common::DECIMAL9))
+		(FromTo(common::FLOAT8, common::DECIMAL18))
+		(FromTo(common::FLOAT8, common::DECIMAL28SPARSE))
+		(FromTo(common::FLOAT8, common::DECIMAL38SPARSE))
+		(FromTo(common::FLOAT8, common::DATE))
+		(FromTo(common::FLOAT8, common::TIME))
+		(FromTo(common::FLOAT8, common::TIMESTAMP))
+		(FromTo(common::FLOAT8, common::INTERVAL))
+		(FromTo(common::FLOAT8, common::FLOAT4))
+		(FromTo(common::FLOAT8, common::FLOAT8))
+		(FromTo(common::FLOAT8, common::BIT))
+		(FromTo(common::FLOAT8, common::VARCHAR))
+		(FromTo(common::FLOAT8, common::VAR16CHAR))
+		(FromTo(common::FLOAT8, common::VARBINARY))
+		(FromTo(common::FLOAT8, common::INTERVALYEAR))
+		(FromTo(common::FLOAT8, common::INTERVALDAY))
+		(FromTo(common::BIT, common::TINYINT))
+		(FromTo(common::BIT, common::INT))
+		(FromTo(common::BIT, common::BIGINT))
+		(FromTo(common::BIT, common::DECIMAL9))
+		(FromTo(common::BIT, common::DECIMAL18))
+		(FromTo(common::BIT, common::DECIMAL28SPARSE))
+		(FromTo(common::BIT, common::DECIMAL38SPARSE))
+		(FromTo(common::BIT, common::DATE))
+		(FromTo(common::BIT, common::TIME))
+		(FromTo(common::BIT, common::TIMESTAMP))
+		(FromTo(common::BIT, common::INTERVAL))
+		(FromTo(common::BIT, common::FLOAT4))
+		(FromTo(common::BIT, common::FLOAT8))
+		(FromTo(common::BIT, common::BIT))
+		(FromTo(common::BIT, common::VARCHAR))
+		(FromTo(common::BIT, common::VAR16CHAR))
+		(FromTo(common::BIT, common::VARBINARY))
+		(FromTo(common::BIT, common::INTERVALYEAR))
+		(FromTo(common::BIT, common::INTERVALDAY))
+		(FromTo(common::FIXEDCHAR, common::TINYINT))
+		(FromTo(common::FIXEDCHAR, common::INT))
+		(FromTo(common::FIXEDCHAR, common::BIGINT))
+		(FromTo(common::FIXEDCHAR, common::DECIMAL9))
+		(FromTo(common::FIXEDCHAR, common::DECIMAL18))
+		(FromTo(common::FIXEDCHAR, common::DECIMAL28SPARSE))
+		(FromTo(common::FIXEDCHAR, common::DECIMAL38SPARSE))
+		(FromTo(common::FIXEDCHAR, common::DATE))
+		(FromTo(common::FIXEDCHAR, common::TIME))
+		(FromTo(common::FIXEDCHAR, common::TIMESTAMP))
+		(FromTo(common::FIXEDCHAR, common::INTERVAL))
+		(FromTo(common::FIXEDCHAR, common::FLOAT4))
+		(FromTo(common::FIXEDCHAR, common::FLOAT8))
+		(FromTo(common::FIXEDCHAR, common::BIT))
+		(FromTo(common::FIXEDCHAR, common::VARCHAR))
+		(FromTo(common::FIXEDCHAR, common::VAR16CHAR))
+		(FromTo(common::FIXEDCHAR, common::VARBINARY))
+		(FromTo(common::FIXEDCHAR, common::INTERVALYEAR))
+		(FromTo(common::FIXEDCHAR, common::INTERVALDAY))
+		(FromTo(common::FIXED16CHAR, common::TINYINT))
+		(FromTo(common::FIXED16CHAR, common::INT))
+		(FromTo(common::FIXED16CHAR, common::BIGINT))
+		(FromTo(common::FIXED16CHAR, common::DECIMAL9))
+		(FromTo(common::FIXED16CHAR, common::DECIMAL18))
+		(FromTo(common::FIXED16CHAR, common::DECIMAL28SPARSE))
+		(FromTo(common::FIXED16CHAR, common::DECIMAL38SPARSE))
+		(FromTo(common::FIXED16CHAR, common::DATE))
+		(FromTo(common::FIXED16CHAR, common::TIME))
+		(FromTo(common::FIXED16CHAR, common::TIMESTAMP))
+		(FromTo(common::FIXED16CHAR, common::INTERVAL))
+		(FromTo(common::FIXED16CHAR, common::FLOAT4))
+		(FromTo(common::FIXED16CHAR, common::FLOAT8))
+		(FromTo(common::FIXED16CHAR, common::BIT))
+		(FromTo(common::FIXED16CHAR, common::VARCHAR))
+		(FromTo(common::FIXED16CHAR, common::VAR16CHAR))
+		(FromTo(common::FIXED16CHAR, common::VARBINARY))
+		(FromTo(common::FIXED16CHAR, common::INTERVALYEAR))
+		(FromTo(common::FIXED16CHAR, common::INTERVALDAY))
+		(FromTo(common::FIXEDBINARY, common::INT))
+		(FromTo(common::FIXEDBINARY, common::BIGINT))
+		(FromTo(common::FIXEDBINARY, common::DECIMAL9))
+		(FromTo(common::FIXEDBINARY, common::DECIMAL18))
+		(FromTo(common::FIXEDBINARY, common::DECIMAL28SPARSE))
+		(FromTo(common::FIXEDBINARY, common::DECIMAL38SPARSE))
+		(FromTo(common::FIXEDBINARY, common::DATE))
+		(FromTo(common::FIXEDBINARY, common::TIME))
+		(FromTo(common::FIXEDBINARY, common::TIMESTAMP))
+		(FromTo(common::FIXEDBINARY, common::INTERVAL))
+		(FromTo(common::FIXEDBINARY, common::FLOAT4))
+		(FromTo(common::FIXEDBINARY, common::FLOAT8))
+		(FromTo(common::FIXEDBINARY, common::BIT))
+		(FromTo(common::FIXEDBINARY, common::VARCHAR))
+		(FromTo(common::FIXEDBINARY, common::VAR16CHAR))
+		(FromTo(common::FIXEDBINARY, common::VARBINARY))
+		(FromTo(common::FIXEDBINARY, common::INTERVALYEAR))
+		(FromTo(common::FIXEDBINARY, common::INTERVALDAY))
+		(FromTo(common::VARCHAR, common::TINYINT))
+		(FromTo(common::VARCHAR, common::INT))
+		(FromTo(common::VARCHAR, common::BIGINT))
+		(FromTo(common::VARCHAR, common::DECIMAL9))
+		(FromTo(common::VARCHAR, common::DECIMAL18))
+		(FromTo(common::VARCHAR, common::DECIMAL28SPARSE))
+		(FromTo(common::VARCHAR, common::DECIMAL38SPARSE))
+		(FromTo(common::VARCHAR, common::DATE))
+		(FromTo(common::VARCHAR, common::TIME))
+		(FromTo(common::VARCHAR, common::TIMESTAMP))
+		(FromTo(common::VARCHAR, common::INTERVAL))
+		(FromTo(common::VARCHAR, common::FLOAT4))
+		(FromTo(common::VARCHAR, common::FLOAT8))
+		(FromTo(common::VARCHAR, common::BIT))
+		(FromTo(common::VARCHAR, common::VARCHAR))
+		(FromTo(common::VARCHAR, common::VAR16CHAR))
+		(FromTo(common::VARCHAR, common::VARBINARY))
+		(FromTo(common::VARCHAR, common::INTERVALYEAR))
+		(FromTo(common::VARCHAR, common::INTERVALDAY))
+		(FromTo(common::VAR16CHAR, common::TINYINT))
+		(FromTo(common::VAR16CHAR, common::INT))
+		(FromTo(common::VAR16CHAR, common::BIGINT))
+		(FromTo(common::VAR16CHAR, common::DECIMAL9))
+		(FromTo(common::VAR16CHAR, common::DECIMAL18))
+		(FromTo(common::VAR16CHAR, common::DECIMAL28SPARSE))
+		(FromTo(common::VAR16CHAR, common::DECIMAL38SPARSE))
+		(FromTo(common::VAR16CHAR, common::DATE))
+		(FromTo(common::VAR16CHAR, common::TIME))
+		(FromTo(common::VAR16CHAR, common::TIMESTAMP))
+		(FromTo(common::VAR16CHAR, common::INTERVAL))
+		(FromTo(common::VAR16CHAR, common::FLOAT4))
+		(FromTo(common::VAR16CHAR, common::FLOAT8))
+		(FromTo(common::VAR16CHAR, common::BIT))
+		(FromTo(common::VAR16CHAR, common::VARCHAR))
+		(FromTo(common::VAR16CHAR, common::VARBINARY))
+		(FromTo(common::VAR16CHAR, common::INTERVALYEAR))
+		(FromTo(common::VAR16CHAR, common::INTERVALDAY))
+		(FromTo(common::VARBINARY, common::TINYINT))
+		(FromTo(common::VARBINARY, common::INT))
+		(FromTo(common::VARBINARY, common::BIGINT))
+		(FromTo(common::VARBINARY, common::DECIMAL9))
+		(FromTo(common::VARBINARY, common::DECIMAL18))
+		(FromTo(common::VARBINARY, common::DECIMAL28SPARSE))
+		(FromTo(common::VARBINARY, common::DECIMAL38SPARSE))
+		(FromTo(common::VARBINARY, common::DATE))
+		(FromTo(common::VARBINARY, common::TIME))
+		(FromTo(common::VARBINARY, common::TIMESTAMP))
+		(FromTo(common::VARBINARY, common::INTERVAL))
+		(FromTo(common::VARBINARY, common::FLOAT4))
+		(FromTo(common::VARBINARY, common::FLOAT8))
+		(FromTo(common::VARBINARY, common::BIT))
+		(FromTo(common::VARBINARY, common::VARCHAR))
+		(FromTo(common::VARBINARY, common::VAR16CHAR))
+		(FromTo(common::VARBINARY, common::VARBINARY))
+		(FromTo(common::VARBINARY, common::INTERVALYEAR))
+		(FromTo(common::VARBINARY, common::INTERVALDAY))
+		(FromTo(common::UINT1, common::INT))
+		(FromTo(common::UINT1, common::BIGINT))
+		(FromTo(common::UINT1, common::DECIMAL9))
+		(FromTo(common::UINT1, common::DECIMAL18))
+		(FromTo(common::UINT1, common::DECIMAL28SPARSE))
+		(FromTo(common::UINT1, common::DECIMAL38SPARSE))
+		(FromTo(common::UINT1, common::DATE))
+		(FromTo(common::UINT1, common::TIME))
+		(FromTo(common::UINT1, common::TIMESTAMP))
+		(FromTo(common::UINT1, common::INTERVAL))
+		(FromTo(common::UINT1, common::FLOAT4))
+		(FromTo(common::UINT1, common::FLOAT8))
+		(FromTo(common::UINT1, common::BIT))
+		(FromTo(common::UINT1, common::VARCHAR))
+		(FromTo(common::UINT1, common::VAR16CHAR))
+		(FromTo(common::UINT1, common::VARBINARY))
+		(FromTo(common::UINT1, common::INTERVALYEAR))
+		(FromTo(common::UINT1, common::INTERVALDAY))
+		(FromTo(common::UINT2, common::INT))
+		(FromTo(common::UINT2, common::BIGINT))
+		(FromTo(common::UINT2, common::DECIMAL9))
+		(FromTo(common::UINT2, common::DECIMAL18))
+		(FromTo(common::UINT2, common::DECIMAL28SPARSE))
+		(FromTo(common::UINT2, common::DECIMAL38SPARSE))
+		(FromTo(common::UINT2, common::DATE))
+		(FromTo(common::UINT2, common::TIME))
+		(FromTo(common::UINT2, common::TIMESTAMP))
+		(FromTo(common::UINT2, common::INTERVAL))
+		(FromTo(common::UINT2, common::FLOAT4))
+		(FromTo(common::UINT2, common::FLOAT8))
+		(FromTo(common::UINT2, common::BIT))
+		(FromTo(common::UINT2, common::VARCHAR))
+		(FromTo(common::UINT2, common::VAR16CHAR))
+		(FromTo(common::UINT2, common::VARBINARY))
+		(FromTo(common::UINT2, common::INTERVALYEAR))
+		(FromTo(common::UINT2, common::INTERVALDAY))
+		(FromTo(common::UINT4, common::INT))
+		(FromTo(common::UINT4, common::BIGINT))
+		(FromTo(common::UINT4, common::DECIMAL9))
+		(FromTo(common::UINT4, common::DECIMAL18))
+		(FromTo(common::UINT4, common::DECIMAL28SPARSE))
+		(FromTo(common::UINT4, common::DECIMAL38SPARSE))
+		(FromTo(common::UINT4, common::DATE))
+		(FromTo(common::UINT4, common::TIME))
+		(FromTo(common::UINT4, common::TIMESTAMP))
+		(FromTo(common::UINT4, common::INTERVAL))
+		(FromTo(common::UINT4, common::FLOAT4))
+		(FromTo(common::UINT4, common::FLOAT8))
+		(FromTo(common::UINT4, common::BIT))
+		(FromTo(common::UINT4, common::VARCHAR))
+		(FromTo(common::UINT4, common::VAR16CHAR))
+		(FromTo(common::UINT4, common::VARBINARY))
+		(FromTo(common::UINT4, common::INTERVALYEAR))
+		(FromTo(common::UINT4, common::INTERVALDAY))
+		(FromTo(common::UINT8, common::INT))
+		(FromTo(common::UINT8, common::BIGINT))
+		(FromTo(common::UINT8, common::DECIMAL9))
+		(FromTo(common::UINT8, common::DECIMAL18))
+		(FromTo(common::UINT8, common::DECIMAL28SPARSE))
+		(FromTo(common::UINT8, common::DECIMAL38SPARSE))
+		(FromTo(common::UINT8, common::DATE))
+		(FromTo(common::UINT8, common::TIME))
+		(FromTo(common::UINT8, common::TIMESTAMP))
+		(FromTo(common::UINT8, common::INTERVAL))
+		(FromTo(common::UINT8, common::FLOAT4))
+		(FromTo(common::UINT8, common::FLOAT8))
+		(FromTo(common::UINT8, common::BIT))
+		(FromTo(common::UINT8, common::VARCHAR))
+		(FromTo(common::UINT8, common::VAR16CHAR))
+		(FromTo(common::UINT8, common::VARBINARY))
+		(FromTo(common::UINT8, common::INTERVALYEAR))
+		(FromTo(common::UINT8, common::INTERVALDAY))
+		(FromTo(common::DECIMAL28DENSE, common::INT))
+		(FromTo(common::DECIMAL28DENSE, common::BIGINT))
+		(FromTo(common::DECIMAL28DENSE, common::DECIMAL9))
+		(FromTo(common::DECIMAL28DENSE, common::DECIMAL18))
+		(FromTo(common::DECIMAL28DENSE, common::DECIMAL28SPARSE))
+		(FromTo(common::DECIMAL28DENSE, common::DECIMAL38SPARSE))
+		(FromTo(common::DECIMAL28DENSE, common::DATE))
+		(FromTo(common::DECIMAL28DENSE, common::TIME))
+		(FromTo(common::DECIMAL28DENSE, common::TIMESTAMP))
+		(FromTo(common::DECIMAL28DENSE, common::INTERVAL))
+		(FromTo(common::DECIMAL28DENSE, common::FLOAT4))
+		(FromTo(common::DECIMAL28DENSE, common::FLOAT8))
+		(FromTo(common::DECIMAL28DENSE, common::BIT))
+		(FromTo(common::DECIMAL28DENSE, common::VARCHAR))
+		(FromTo(common::DECIMAL28DENSE, common::VAR16CHAR))
+		(FromTo(common::DECIMAL28DENSE, common::VARBINARY))
+		(FromTo(common::DECIMAL28DENSE, common::INTERVALYEAR))
+		(FromTo(common::DECIMAL28DENSE, common::INTERVALDAY))
+		(FromTo(common::DECIMAL38DENSE, common::INT))
+		(FromTo(common::DECIMAL38DENSE, common::BIGINT))
+		(FromTo(common::DECIMAL38DENSE, common::DECIMAL9))
+		(FromTo(common::DECIMAL38DENSE, common::DECIMAL18))
+		(FromTo(common::DECIMAL38DENSE, common::DECIMAL28SPARSE))
+		(FromTo(common::DECIMAL38DENSE, common::DECIMAL38SPARSE))
+		(FromTo(common::DECIMAL38DENSE, common::DATE))
+		(FromTo(common::DECIMAL38DENSE, common::TIME))
+		(FromTo(common::DECIMAL38DENSE, common::TIMESTAMP))
+		(FromTo(common::DECIMAL38DENSE, common::INTERVAL))
+		(FromTo(common::DECIMAL38DENSE, common::FLOAT4))
+		(FromTo(common::DECIMAL38DENSE, common::FLOAT8))
+		(FromTo(common::DECIMAL38DENSE, common::BIT))
+		(FromTo(common::DECIMAL38DENSE, common::VARCHAR))
+		(FromTo(common::DECIMAL38DENSE, common::VAR16CHAR))
+		(FromTo(common::DECIMAL38DENSE, common::VARBINARY))
+		(FromTo(common::DECIMAL38DENSE, common::INTERVALYEAR))
+		(FromTo(common::DECIMAL38DENSE, common::INTERVALDAY))
+		(FromTo(common::DM_UNKNOWN, common::TINYINT))
+		(FromTo(common::DM_UNKNOWN, common::INT))
+		(FromTo(common::DM_UNKNOWN, common::BIGINT))
+		(FromTo(common::DM_UNKNOWN, common::DECIMAL9))
+		(FromTo(common::DM_UNKNOWN, common::DECIMAL18))
+		(FromTo(common::DM_UNKNOWN, common::DECIMAL28SPARSE))
+		(FromTo(common::DM_UNKNOWN, common::DECIMAL38SPARSE))
+		(FromTo(common::DM_UNKNOWN, common::DATE))
+		(FromTo(common::DM_UNKNOWN, common::TIME))
+		(FromTo(common::DM_UNKNOWN, common::TIMESTAMP))
+		(FromTo(common::DM_UNKNOWN, common::INTERVAL))
+		(FromTo(common::DM_UNKNOWN, common::FLOAT4))
+		(FromTo(common::DM_UNKNOWN, common::FLOAT8))
+		(FromTo(common::DM_UNKNOWN, common::BIT))
+		(FromTo(common::DM_UNKNOWN, common::VARCHAR))
+		(FromTo(common::DM_UNKNOWN, common::VAR16CHAR))
+		(FromTo(common::DM_UNKNOWN, common::VARBINARY))
+		(FromTo(common::DM_UNKNOWN, common::INTERVALYEAR))
+		(FromTo(common::DM_UNKNOWN, common::INTERVALDAY))
+		(FromTo(common::INTERVALYEAR, common::INT))
+		(FromTo(common::INTERVALYEAR, common::BIGINT))
+		(FromTo(common::INTERVALYEAR, common::DECIMAL9))
+		(FromTo(common::INTERVALYEAR, common::DECIMAL18))
+		(FromTo(common::INTERVALYEAR, common::DECIMAL28SPARSE))
+		(FromTo(common::INTERVALYEAR, common::DECIMAL38SPARSE))
+		(FromTo(common::INTERVALYEAR, common::DATE))
+		(FromTo(common::INTERVALYEAR, common::TIME))
+		(FromTo(common::INTERVALYEAR, common::TIMESTAMP))
+		(FromTo(common::INTERVALYEAR, common::INTERVAL))
+		(FromTo(common::INTERVALYEAR, common::FLOAT4))
+		(FromTo(common::INTERVALYEAR, common::FLOAT8))
+		(FromTo(common::INTERVALYEAR, common::BIT))
+		(FromTo(common::INTERVALYEAR, common::VARCHAR))
+		(FromTo(common::INTERVALYEAR, common::VAR16CHAR))
+		(FromTo(common::INTERVALYEAR, common::VARBINARY))
+		(FromTo(common::INTERVALYEAR, common::INTERVALYEAR))
+		(FromTo(common::INTERVALYEAR, common::INTERVALDAY))
+		(FromTo(common::INTERVALDAY, common::INT))
+		(FromTo(common::INTERVALDAY, common::BIGINT))
+		(FromTo(common::INTERVALDAY, common::DECIMAL9))
+		(FromTo(common::INTERVALDAY, common::DECIMAL18))
+		(FromTo(common::INTERVALDAY, common::DECIMAL28SPARSE))
+		(FromTo(common::INTERVALDAY, common::DECIMAL38SPARSE))
+		(FromTo(common::INTERVALDAY, common::DATE))
+		(FromTo(common::INTERVALDAY, common::TIME))
+		(FromTo(common::INTERVALDAY, common::TIMESTAMP))
+		(FromTo(common::INTERVALDAY, common::INTERVAL))
+		(FromTo(common::INTERVALDAY, common::FLOAT4))
+		(FromTo(common::INTERVALDAY, common::FLOAT8))
+		(FromTo(common::INTERVALDAY, common::BIT))
+		(FromTo(common::INTERVALDAY, common::VARCHAR))
+		(FromTo(common::INTERVALDAY, common::VAR16CHAR))
+		(FromTo(common::INTERVALDAY, common::VARBINARY))
+		(FromTo(common::INTERVALDAY, common::INTERVALYEAR))
+		(FromTo(common::INTERVALDAY, common::INTERVALDAY));
+} // anonymous namespace
+
+// Conversion scalar function support
+bool DrillMetadata::isConvertSupported(common::MinorType from, common::MinorType to) const {
+	return s_convertMap.find(FromTo(from,to)) != s_convertMap.end();
+}
+
+const std::string& DrillMetadata::getServerName() const {
+	return m_client.getServerInfos().name();
+}
+const std::string& DrillMetadata::getServerVersion() const {
+	return m_client.getServerInfos().version();
+}
+uint32_t DrillMetadata::getServerMajorVersion() const {
+	return m_client.getServerInfos().majorversion();
+}
+
+uint32_t DrillMetadata::getServerMinorVersion() const {
+	return m_client.getServerInfos().minorversion();
+}
+
+uint32_t DrillMetadata::getServerPatchVersion() const {
+	return m_client.getServerInfos().patchversion();
+}
+
+status_t DrillMetadata::getCatalogs(const std::string& catalogPattern, Metadata::pfnCatalogMetadataListener listener, void* listenerCtx, QueryHandle_t* qHandle) {
+	DrillClientCatalogResult* result = m_client.getCatalogs(catalogPattern, listener, listenerCtx);
+	if(result==NULL){
+		*qHandle=NULL;
+		return static_cast<status_t>(m_client.getError()->status);
+	}
+	*qHandle=reinterpret_cast<QueryHandle_t>(result);
+	return QRY_SUCCESS;
+}
+status_t DrillMetadata::getSchemas(const std::string& catalogPattern, const std::string& schemaPattern, Metadata::pfnSchemaMetadataListener listener, void* listenerCtx, QueryHandle_t* qHandle) {
+	DrillClientSchemaResult* result = m_client.getSchemas(catalogPattern, schemaPattern, listener, listenerCtx);
+	if(result==NULL){
+		*qHandle=NULL;
+		return static_cast<status_t>(m_client.getError()->status);
+	}
+	*qHandle=reinterpret_cast<QueryHandle_t>(result);
+	return QRY_SUCCESS;
+}
+status_t DrillMetadata::getTables(const std::string& catalogPattern, const std::string& schemaPattern, const std::string& tablePattern, const std::vector<std::string>* tableTypes, Metadata::pfnTableMetadataListener listener, void* listenerCtx, QueryHandle_t* qHandle) {
+	DrillClientTableResult* result = m_client.getTables(catalogPattern, schemaPattern, tablePattern, tableTypes, listener, listenerCtx);
+	if(result==NULL){
+		*qHandle=NULL;
+		return static_cast<status_t>(m_client.getError()->status);
+	}
+	*qHandle=reinterpret_cast<QueryHandle_t>(result);
+	return QRY_SUCCESS;
+}
+status_t DrillMetadata::getColumns(const std::string& catalogPattern, const std::string& schemaPattern, const std:: string& tablePattern, const std::string& columnPattern, Metadata::pfnColumnMetadataListener listener, void* listenerCtx, QueryHandle_t* qHandle) {
+	DrillClientColumnResult* result = m_client.getColumns(catalogPattern, schemaPattern, tablePattern, columnPattern, listener, listenerCtx);
+	if(result==NULL){
+		*qHandle=NULL;
+		return static_cast<status_t>(m_client.getError()->status);
+	}
+	*qHandle=reinterpret_cast<QueryHandle_t>(result);
+	return QRY_SUCCESS;
+}
+} // namespace meta
+} // namespace Drill


[09/15] drill git commit: DRILL-4853: Update C++ protobuf source files

Posted by pa...@apache.org.
DRILL-4853: Update C++ protobuf source files

Add support for prepared statements and metadata querying


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

Branch: refs/heads/master
Commit: 2558803ecdfc961bb630e9e2372255c44f986d06
Parents: 7e7214b
Author: Laurent Goujon <la...@dremio.com>
Authored: Thu Aug 18 09:55:59 2016 -0700
Committer: Parth Chandra <pa...@apache.org>
Committed: Tue Nov 1 11:29:58 2016 -0700

----------------------------------------------------------------------
 contrib/native/client/readme.linux              |    10 +
 contrib/native/client/scripts/fixProtodefs.sh   |    20 +-
 .../native/client/src/protobuf/CMakeLists.txt   |     2 +-
 .../client/src/protobuf/ExecutionProtos.pb.cc   |   310 +-
 .../client/src/protobuf/ExecutionProtos.pb.h    |   214 +-
 contrib/native/client/src/protobuf/User.pb.cc   | 10122 ++++++++++++++++-
 contrib/native/client/src/protobuf/User.pb.h    |  8099 ++++++++++++-
 .../client/src/protobuf/UserBitShared.pb.cc     |   583 +-
 .../client/src/protobuf/UserBitShared.pb.h      |   344 +-
 9 files changed, 18543 insertions(+), 1161 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/2558803e/contrib/native/client/readme.linux
----------------------------------------------------------------------
diff --git a/contrib/native/client/readme.linux b/contrib/native/client/readme.linux
index 3e2336e..4eaeea5 100644
--- a/contrib/native/client/readme.linux
+++ b/contrib/native/client/readme.linux
@@ -84,6 +84,16 @@ OR
     ln -svf libboost_filesystem.a libboost_filesystem-mt.a
     ln -svf libboost_date_time.a libboost_date_time-mt.a
 
+(Optional) Refresh protobuf source files
+----------------------------------------
+When changes have been introduced to the protocol module, you might need to refresh the protobuf C++ source files too.
+    $> cd DRILL_DIR/contrib/native/client
+    $> mkdir build
+    $> cd build && cmake3 -G "Unix Makefiles" ..
+    $> make cpProtobufs
+
+Open a pull request with the changes to DRILL_DIR/contrib/native/client/src/protobuf
+
 Build drill client
 -------------------
     $> cd DRILL_DIR/contrib/native/client

http://git-wip-us.apache.org/repos/asf/drill/blob/2558803e/contrib/native/client/scripts/fixProtodefs.sh
----------------------------------------------------------------------
diff --git a/contrib/native/client/scripts/fixProtodefs.sh b/contrib/native/client/scripts/fixProtodefs.sh
index f3ce781..d882ca6 100755
--- a/contrib/native/client/scripts/fixProtodefs.sh
+++ b/contrib/native/client/scripts/fixProtodefs.sh
@@ -36,21 +36,15 @@ main() {
     then
         echo "Creating Protobuf directory"
         mkdir -p ${TARGDIR}
-        cp -r ${SRCDIR}/* ${TARGDIR}
+    fi
+    cp -r ${SRCDIR}/* ${TARGDIR}
+
+    if [ -e ${TARGDIR}/${FNAME} ]
+    then
         fixFile ${FNAME}
     else
-        cp -r ${SRCDIR}/* ${TARGDIR}
-
-        if [ -e ${TARGDIR}/${FNAME} ]
-        then
-            if [ ${SRCDIR}/${FNAME} -nt ${TARGDIR}/${FNAME} ]
-            then
-                fixFile ${FNAME}
-            fi
-        else
-            echo "$FNAME not found"
-            exit 1
-        fi
+        echo "$FNAME not found"
+        exit 1
     fi
 }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/2558803e/contrib/native/client/src/protobuf/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/protobuf/CMakeLists.txt b/contrib/native/client/src/protobuf/CMakeLists.txt
index 474d2f6..edae72f 100644
--- a/contrib/native/client/src/protobuf/CMakeLists.txt
+++ b/contrib/native/client/src/protobuf/CMakeLists.txt
@@ -92,7 +92,7 @@ add_custom_target(fixProtobufs
     COMMAND ${PROJECT_SOURCE_DIR}/scripts/fixProtodefs.sh ${SRCDIR} ${TARGDIR} ${FNAME}
     )
 
-add_custom_target(genProtobufs DEPENDS ${GenProtoSources})
+add_custom_target(genProtobufs DEPENDS ${GenProtoSources} fixProtobufs)
 
 # copy protobuf CPP files to destinations in the source tree
 add_custom_target(cpProtobufs

http://git-wip-us.apache.org/repos/asf/drill/blob/2558803e/contrib/native/client/src/protobuf/ExecutionProtos.pb.cc
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/protobuf/ExecutionProtos.pb.cc b/contrib/native/client/src/protobuf/ExecutionProtos.pb.cc
index a36bd9f..26690f3 100644
--- a/contrib/native/client/src/protobuf/ExecutionProtos.pb.cc
+++ b/contrib/native/client/src/protobuf/ExecutionProtos.pb.cc
@@ -24,6 +24,9 @@ namespace {
 const ::google::protobuf::Descriptor* FragmentHandle_descriptor_ = NULL;
 const ::google::protobuf::internal::GeneratedMessageReflection*
   FragmentHandle_reflection_ = NULL;
+const ::google::protobuf::Descriptor* ServerPreparedStatementState_descriptor_ = NULL;
+const ::google::protobuf::internal::GeneratedMessageReflection*
+  ServerPreparedStatementState_reflection_ = NULL;
 
 }  // namespace
 
@@ -35,10 +38,11 @@ void protobuf_AssignDesc_ExecutionProtos_2eproto() {
       "ExecutionProtos.proto");
   GOOGLE_CHECK(file != NULL);
   FragmentHandle_descriptor_ = file->message_type(0);
-  static const int FragmentHandle_offsets_[3] = {
+  static const int FragmentHandle_offsets_[4] = {
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(FragmentHandle, query_id_),
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(FragmentHandle, major_fragment_id_),
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(FragmentHandle, minor_fragment_id_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(FragmentHandle, parent_query_id_),
   };
   FragmentHandle_reflection_ =
     new ::google::protobuf::internal::GeneratedMessageReflection(
@@ -51,6 +55,21 @@ void protobuf_AssignDesc_ExecutionProtos_2eproto() {
       ::google::protobuf::DescriptorPool::generated_pool(),
       ::google::protobuf::MessageFactory::generated_factory(),
       sizeof(FragmentHandle));
+  ServerPreparedStatementState_descriptor_ = file->message_type(1);
+  static const int ServerPreparedStatementState_offsets_[1] = {
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ServerPreparedStatementState, sql_query_),
+  };
+  ServerPreparedStatementState_reflection_ =
+    new ::google::protobuf::internal::GeneratedMessageReflection(
+      ServerPreparedStatementState_descriptor_,
+      ServerPreparedStatementState::default_instance_,
+      ServerPreparedStatementState_offsets_,
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ServerPreparedStatementState, _has_bits_[0]),
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ServerPreparedStatementState, _unknown_fields_),
+      -1,
+      ::google::protobuf::DescriptorPool::generated_pool(),
+      ::google::protobuf::MessageFactory::generated_factory(),
+      sizeof(ServerPreparedStatementState));
 }
 
 namespace {
@@ -65,6 +84,8 @@ void protobuf_RegisterTypes(const ::std::string&) {
   protobuf_AssignDescriptorsOnce();
   ::google::protobuf::MessageFactory::InternalRegisterGeneratedMessage(
     FragmentHandle_descriptor_, &FragmentHandle::default_instance());
+  ::google::protobuf::MessageFactory::InternalRegisterGeneratedMessage(
+    ServerPreparedStatementState_descriptor_, &ServerPreparedStatementState::default_instance());
 }
 
 }  // namespace
@@ -72,6 +93,8 @@ void protobuf_RegisterTypes(const ::std::string&) {
 void protobuf_ShutdownFile_ExecutionProtos_2eproto() {
   delete FragmentHandle::default_instance_;
   delete FragmentHandle_reflection_;
+  delete ServerPreparedStatementState::default_instance_;
+  delete ServerPreparedStatementState_reflection_;
 }
 
 void protobuf_AddDesc_ExecutionProtos_2eproto() {
@@ -84,15 +107,20 @@ void protobuf_AddDesc_ExecutionProtos_2eproto() {
   ::exec::shared::protobuf_AddDesc_UserBitShared_2eproto();
   ::google::protobuf::DescriptorPool::InternalAddGeneratedFile(
     "\n\025ExecutionProtos.proto\022\010exec.bit\032\022Coord"
-    "ination.proto\032\023UserBitShared.proto\"n\n\016Fr"
-    "agmentHandle\022&\n\010query_id\030\001 \001(\0132\024.exec.sh"
-    "ared.QueryId\022\031\n\021major_fragment_id\030\002 \001(\005\022"
-    "\031\n\021minor_fragment_id\030\003 \001(\005B+\n\033org.apache"
-    ".drill.exec.protoB\nExecProtosH\001", 231);
+    "ination.proto\032\023UserBitShared.proto\"\235\001\n\016F"
+    "ragmentHandle\022&\n\010query_id\030\001 \001(\0132\024.exec.s"
+    "hared.QueryId\022\031\n\021major_fragment_id\030\002 \001(\005"
+    "\022\031\n\021minor_fragment_id\030\003 \001(\005\022-\n\017parent_qu"
+    "ery_id\030\004 \001(\0132\024.exec.shared.QueryId\"1\n\034Se"
+    "rverPreparedStatementState\022\021\n\tsql_query\030"
+    "\001 \001(\tB+\n\033org.apache.drill.exec.protoB\nEx"
+    "ecProtosH\001", 330);
   ::google::protobuf::MessageFactory::InternalRegisterGeneratedFile(
     "ExecutionProtos.proto", &protobuf_RegisterTypes);
   FragmentHandle::default_instance_ = new FragmentHandle();
+  ServerPreparedStatementState::default_instance_ = new ServerPreparedStatementState();
   FragmentHandle::default_instance_->InitAsDefaultInstance();
+  ServerPreparedStatementState::default_instance_->InitAsDefaultInstance();
   ::google::protobuf::internal::OnShutdown(&protobuf_ShutdownFile_ExecutionProtos_2eproto);
 }
 
@@ -109,6 +137,7 @@ struct StaticDescriptorInitializer_ExecutionProtos_2eproto {
 const int FragmentHandle::kQueryIdFieldNumber;
 const int FragmentHandle::kMajorFragmentIdFieldNumber;
 const int FragmentHandle::kMinorFragmentIdFieldNumber;
+const int FragmentHandle::kParentQueryIdFieldNumber;
 #endif  // !_MSC_VER
 
 FragmentHandle::FragmentHandle()
@@ -118,6 +147,7 @@ FragmentHandle::FragmentHandle()
 
 void FragmentHandle::InitAsDefaultInstance() {
   query_id_ = const_cast< ::exec::shared::QueryId*>(&::exec::shared::QueryId::default_instance());
+  parent_query_id_ = const_cast< ::exec::shared::QueryId*>(&::exec::shared::QueryId::default_instance());
 }
 
 FragmentHandle::FragmentHandle(const FragmentHandle& from)
@@ -131,6 +161,7 @@ void FragmentHandle::SharedCtor() {
   query_id_ = NULL;
   major_fragment_id_ = 0;
   minor_fragment_id_ = 0;
+  parent_query_id_ = NULL;
   ::memset(_has_bits_, 0, sizeof(_has_bits_));
 }
 
@@ -141,6 +172,7 @@ FragmentHandle::~FragmentHandle() {
 void FragmentHandle::SharedDtor() {
   if (this != default_instance_) {
     delete query_id_;
+    delete parent_query_id_;
   }
 }
 
@@ -172,6 +204,9 @@ void FragmentHandle::Clear() {
     }
     major_fragment_id_ = 0;
     minor_fragment_id_ = 0;
+    if (has_parent_query_id()) {
+      if (parent_query_id_ != NULL) parent_query_id_->::exec::shared::QueryId::Clear();
+    }
   }
   ::memset(_has_bits_, 0, sizeof(_has_bits_));
   mutable_unknown_fields()->Clear();
@@ -224,6 +259,20 @@ bool FragmentHandle::MergePartialFromCodedStream(
         } else {
           goto handle_uninterpreted;
         }
+        if (input->ExpectTag(34)) goto parse_parent_query_id;
+        break;
+      }
+
+      // optional .exec.shared.QueryId parent_query_id = 4;
+      case 4: {
+        if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
+            ::google::protobuf::internal::WireFormatLite::WIRETYPE_LENGTH_DELIMITED) {
+         parse_parent_query_id:
+          DO_(::google::protobuf::internal::WireFormatLite::ReadMessageNoVirtual(
+               input, mutable_parent_query_id()));
+        } else {
+          goto handle_uninterpreted;
+        }
         if (input->ExpectAtEnd()) return true;
         break;
       }
@@ -262,6 +311,12 @@ void FragmentHandle::SerializeWithCachedSizes(
     ::google::protobuf::internal::WireFormatLite::WriteInt32(3, this->minor_fragment_id(), output);
   }
 
+  // optional .exec.shared.QueryId parent_query_id = 4;
+  if (has_parent_query_id()) {
+    ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray(
+      4, this->parent_query_id(), output);
+  }
+
   if (!unknown_fields().empty()) {
     ::google::protobuf::internal::WireFormat::SerializeUnknownFields(
         unknown_fields(), output);
@@ -287,6 +342,13 @@ void FragmentHandle::SerializeWithCachedSizes(
     target = ::google::protobuf::internal::WireFormatLite::WriteInt32ToArray(3, this->minor_fragment_id(), target);
   }
 
+  // optional .exec.shared.QueryId parent_query_id = 4;
+  if (has_parent_query_id()) {
+    target = ::google::protobuf::internal::WireFormatLite::
+      WriteMessageNoVirtualToArray(
+        4, this->parent_query_id(), target);
+  }
+
   if (!unknown_fields().empty()) {
     target = ::google::protobuf::internal::WireFormat::SerializeUnknownFieldsToArray(
         unknown_fields(), target);
@@ -319,6 +381,13 @@ int FragmentHandle::ByteSize() const {
           this->minor_fragment_id());
     }
 
+    // optional .exec.shared.QueryId parent_query_id = 4;
+    if (has_parent_query_id()) {
+      total_size += 1 +
+        ::google::protobuf::internal::WireFormatLite::MessageSizeNoVirtual(
+          this->parent_query_id());
+    }
+
   }
   if (!unknown_fields().empty()) {
     total_size +=
@@ -355,6 +424,9 @@ void FragmentHandle::MergeFrom(const FragmentHandle& from) {
     if (from.has_minor_fragment_id()) {
       set_minor_fragment_id(from.minor_fragment_id());
     }
+    if (from.has_parent_query_id()) {
+      mutable_parent_query_id()->::exec::shared::QueryId::MergeFrom(from.parent_query_id());
+    }
   }
   mutable_unknown_fields()->MergeFrom(from.unknown_fields());
 }
@@ -381,6 +453,7 @@ void FragmentHandle::Swap(FragmentHandle* other) {
     std::swap(query_id_, other->query_id_);
     std::swap(major_fragment_id_, other->major_fragment_id_);
     std::swap(minor_fragment_id_, other->minor_fragment_id_);
+    std::swap(parent_query_id_, other->parent_query_id_);
     std::swap(_has_bits_[0], other->_has_bits_[0]);
     _unknown_fields_.Swap(&other->_unknown_fields_);
     std::swap(_cached_size_, other->_cached_size_);
@@ -396,6 +469,231 @@ void FragmentHandle::Swap(FragmentHandle* other) {
 }
 
 
+// ===================================================================
+
+#ifndef _MSC_VER
+const int ServerPreparedStatementState::kSqlQueryFieldNumber;
+#endif  // !_MSC_VER
+
+ServerPreparedStatementState::ServerPreparedStatementState()
+  : ::google::protobuf::Message() {
+  SharedCtor();
+}
+
+void ServerPreparedStatementState::InitAsDefaultInstance() {
+}
+
+ServerPreparedStatementState::ServerPreparedStatementState(const ServerPreparedStatementState& from)
+  : ::google::protobuf::Message() {
+  SharedCtor();
+  MergeFrom(from);
+}
+
+void ServerPreparedStatementState::SharedCtor() {
+  _cached_size_ = 0;
+  sql_query_ = const_cast< ::std::string*>(&::google::protobuf::internal::kEmptyString);
+  ::memset(_has_bits_, 0, sizeof(_has_bits_));
+}
+
+ServerPreparedStatementState::~ServerPreparedStatementState() {
+  SharedDtor();
+}
+
+void ServerPreparedStatementState::SharedDtor() {
+  if (sql_query_ != &::google::protobuf::internal::kEmptyString) {
+    delete sql_query_;
+  }
+  if (this != default_instance_) {
+  }
+}
+
+void ServerPreparedStatementState::SetCachedSize(int size) const {
+  GOOGLE_SAFE_CONCURRENT_WRITES_BEGIN();
+  _cached_size_ = size;
+  GOOGLE_SAFE_CONCURRENT_WRITES_END();
+}
+const ::google::protobuf::Descriptor* ServerPreparedStatementState::descriptor() {
+  protobuf_AssignDescriptorsOnce();
+  return ServerPreparedStatementState_descriptor_;
+}
+
+const ServerPreparedStatementState& ServerPreparedStatementState::default_instance() {
+  if (default_instance_ == NULL) protobuf_AddDesc_ExecutionProtos_2eproto();
+  return *default_instance_;
+}
+
+ServerPreparedStatementState* ServerPreparedStatementState::default_instance_ = NULL;
+
+ServerPreparedStatementState* ServerPreparedStatementState::New() const {
+  return new ServerPreparedStatementState;
+}
+
+void ServerPreparedStatementState::Clear() {
+  if (_has_bits_[0 / 32] & (0xffu << (0 % 32))) {
+    if (has_sql_query()) {
+      if (sql_query_ != &::google::protobuf::internal::kEmptyString) {
+        sql_query_->clear();
+      }
+    }
+  }
+  ::memset(_has_bits_, 0, sizeof(_has_bits_));
+  mutable_unknown_fields()->Clear();
+}
+
+bool ServerPreparedStatementState::MergePartialFromCodedStream(
+    ::google::protobuf::io::CodedInputStream* input) {
+#define DO_(EXPRESSION) if (!(EXPRESSION)) return false
+  ::google::protobuf::uint32 tag;
+  while ((tag = input->ReadTag()) != 0) {
+    switch (::google::protobuf::internal::WireFormatLite::GetTagFieldNumber(tag)) {
+      // optional string sql_query = 1;
+      case 1: {
+        if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
+            ::google::protobuf::internal::WireFormatLite::WIRETYPE_LENGTH_DELIMITED) {
+          DO_(::google::protobuf::internal::WireFormatLite::ReadString(
+                input, this->mutable_sql_query()));
+          ::google::protobuf::internal::WireFormat::VerifyUTF8String(
+            this->sql_query().data(), this->sql_query().length(),
+            ::google::protobuf::internal::WireFormat::PARSE);
+        } else {
+          goto handle_uninterpreted;
+        }
+        if (input->ExpectAtEnd()) return true;
+        break;
+      }
+
+      default: {
+      handle_uninterpreted:
+        if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
+            ::google::protobuf::internal::WireFormatLite::WIRETYPE_END_GROUP) {
+          return true;
+        }
+        DO_(::google::protobuf::internal::WireFormat::SkipField(
+              input, tag, mutable_unknown_fields()));
+        break;
+      }
+    }
+  }
+  return true;
+#undef DO_
+}
+
+void ServerPreparedStatementState::SerializeWithCachedSizes(
+    ::google::protobuf::io::CodedOutputStream* output) const {
+  // optional string sql_query = 1;
+  if (has_sql_query()) {
+    ::google::protobuf::internal::WireFormat::VerifyUTF8String(
+      this->sql_query().data(), this->sql_query().length(),
+      ::google::protobuf::internal::WireFormat::SERIALIZE);
+    ::google::protobuf::internal::WireFormatLite::WriteString(
+      1, this->sql_query(), output);
+  }
+
+  if (!unknown_fields().empty()) {
+    ::google::protobuf::internal::WireFormat::SerializeUnknownFields(
+        unknown_fields(), output);
+  }
+}
+
+::google::protobuf::uint8* ServerPreparedStatementState::SerializeWithCachedSizesToArray(
+    ::google::protobuf::uint8* target) const {
+  // optional string sql_query = 1;
+  if (has_sql_query()) {
+    ::google::protobuf::internal::WireFormat::VerifyUTF8String(
+      this->sql_query().data(), this->sql_query().length(),
+      ::google::protobuf::internal::WireFormat::SERIALIZE);
+    target =
+      ::google::protobuf::internal::WireFormatLite::WriteStringToArray(
+        1, this->sql_query(), target);
+  }
+
+  if (!unknown_fields().empty()) {
+    target = ::google::protobuf::internal::WireFormat::SerializeUnknownFieldsToArray(
+        unknown_fields(), target);
+  }
+  return target;
+}
+
+int ServerPreparedStatementState::ByteSize() const {
+  int total_size = 0;
+
+  if (_has_bits_[0 / 32] & (0xffu << (0 % 32))) {
+    // optional string sql_query = 1;
+    if (has_sql_query()) {
+      total_size += 1 +
+        ::google::protobuf::internal::WireFormatLite::StringSize(
+          this->sql_query());
+    }
+
+  }
+  if (!unknown_fields().empty()) {
+    total_size +=
+      ::google::protobuf::internal::WireFormat::ComputeUnknownFieldsSize(
+        unknown_fields());
+  }
+  GOOGLE_SAFE_CONCURRENT_WRITES_BEGIN();
+  _cached_size_ = total_size;
+  GOOGLE_SAFE_CONCURRENT_WRITES_END();
+  return total_size;
+}
+
+void ServerPreparedStatementState::MergeFrom(const ::google::protobuf::Message& from) {
+  GOOGLE_CHECK_NE(&from, this);
+  const ServerPreparedStatementState* source =
+    ::google::protobuf::internal::dynamic_cast_if_available<const ServerPreparedStatementState*>(
+      &from);
+  if (source == NULL) {
+    ::google::protobuf::internal::ReflectionOps::Merge(from, this);
+  } else {
+    MergeFrom(*source);
+  }
+}
+
+void ServerPreparedStatementState::MergeFrom(const ServerPreparedStatementState& from) {
+  GOOGLE_CHECK_NE(&from, this);
+  if (from._has_bits_[0 / 32] & (0xffu << (0 % 32))) {
+    if (from.has_sql_query()) {
+      set_sql_query(from.sql_query());
+    }
+  }
+  mutable_unknown_fields()->MergeFrom(from.unknown_fields());
+}
+
+void ServerPreparedStatementState::CopyFrom(const ::google::protobuf::Message& from) {
+  if (&from == this) return;
+  Clear();
+  MergeFrom(from);
+}
+
+void ServerPreparedStatementState::CopyFrom(const ServerPreparedStatementState& from) {
+  if (&from == this) return;
+  Clear();
+  MergeFrom(from);
+}
+
+bool ServerPreparedStatementState::IsInitialized() const {
+
+  return true;
+}
+
+void ServerPreparedStatementState::Swap(ServerPreparedStatementState* other) {
+  if (other != this) {
+    std::swap(sql_query_, other->sql_query_);
+    std::swap(_has_bits_[0], other->_has_bits_[0]);
+    _unknown_fields_.Swap(&other->_unknown_fields_);
+    std::swap(_cached_size_, other->_cached_size_);
+  }
+}
+
+::google::protobuf::Metadata ServerPreparedStatementState::GetMetadata() const {
+  protobuf_AssignDescriptorsOnce();
+  ::google::protobuf::Metadata metadata;
+  metadata.descriptor = ServerPreparedStatementState_descriptor_;
+  metadata.reflection = ServerPreparedStatementState_reflection_;
+  return metadata;
+}
+
+
 // @@protoc_insertion_point(namespace_scope)
 
 }  // namespace bit

http://git-wip-us.apache.org/repos/asf/drill/blob/2558803e/contrib/native/client/src/protobuf/ExecutionProtos.pb.h
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/protobuf/ExecutionProtos.pb.h b/contrib/native/client/src/protobuf/ExecutionProtos.pb.h
index a602c1f..86cddb9 100644
--- a/contrib/native/client/src/protobuf/ExecutionProtos.pb.h
+++ b/contrib/native/client/src/protobuf/ExecutionProtos.pb.h
@@ -37,6 +37,7 @@ void protobuf_AssignDesc_ExecutionProtos_2eproto();
 void protobuf_ShutdownFile_ExecutionProtos_2eproto();
 
 class FragmentHandle;
+class ServerPreparedStatementState;
 
 // ===================================================================
 
@@ -117,6 +118,15 @@ class FragmentHandle : public ::google::protobuf::Message {
   inline ::google::protobuf::int32 minor_fragment_id() const;
   inline void set_minor_fragment_id(::google::protobuf::int32 value);
 
+  // optional .exec.shared.QueryId parent_query_id = 4;
+  inline bool has_parent_query_id() const;
+  inline void clear_parent_query_id();
+  static const int kParentQueryIdFieldNumber = 4;
+  inline const ::exec::shared::QueryId& parent_query_id() const;
+  inline ::exec::shared::QueryId* mutable_parent_query_id();
+  inline ::exec::shared::QueryId* release_parent_query_id();
+  inline void set_allocated_parent_query_id(::exec::shared::QueryId* parent_query_id);
+
   // @@protoc_insertion_point(class_scope:exec.bit.FragmentHandle)
  private:
   inline void set_has_query_id();
@@ -125,15 +135,18 @@ class FragmentHandle : public ::google::protobuf::Message {
   inline void clear_has_major_fragment_id();
   inline void set_has_minor_fragment_id();
   inline void clear_has_minor_fragment_id();
+  inline void set_has_parent_query_id();
+  inline void clear_has_parent_query_id();
 
   ::google::protobuf::UnknownFieldSet _unknown_fields_;
 
   ::exec::shared::QueryId* query_id_;
   ::google::protobuf::int32 major_fragment_id_;
   ::google::protobuf::int32 minor_fragment_id_;
+  ::exec::shared::QueryId* parent_query_id_;
 
   mutable int _cached_size_;
-  ::google::protobuf::uint32 _has_bits_[(3 + 31) / 32];
+  ::google::protobuf::uint32 _has_bits_[(4 + 31) / 32];
 
   friend void  protobuf_AddDesc_ExecutionProtos_2eproto();
   friend void protobuf_AssignDesc_ExecutionProtos_2eproto();
@@ -142,6 +155,93 @@ class FragmentHandle : public ::google::protobuf::Message {
   void InitAsDefaultInstance();
   static FragmentHandle* default_instance_;
 };
+// -------------------------------------------------------------------
+
+class ServerPreparedStatementState : public ::google::protobuf::Message {
+ public:
+  ServerPreparedStatementState();
+  virtual ~ServerPreparedStatementState();
+
+  ServerPreparedStatementState(const ServerPreparedStatementState& from);
+
+  inline ServerPreparedStatementState& operator=(const ServerPreparedStatementState& from) {
+    CopyFrom(from);
+    return *this;
+  }
+
+  inline const ::google::protobuf::UnknownFieldSet& unknown_fields() const {
+    return _unknown_fields_;
+  }
+
+  inline ::google::protobuf::UnknownFieldSet* mutable_unknown_fields() {
+    return &_unknown_fields_;
+  }
+
+  static const ::google::protobuf::Descriptor* descriptor();
+  static const ServerPreparedStatementState& default_instance();
+
+  void Swap(ServerPreparedStatementState* other);
+
+  // implements Message ----------------------------------------------
+
+  ServerPreparedStatementState* New() const;
+  void CopyFrom(const ::google::protobuf::Message& from);
+  void MergeFrom(const ::google::protobuf::Message& from);
+  void CopyFrom(const ServerPreparedStatementState& from);
+  void MergeFrom(const ServerPreparedStatementState& from);
+  void Clear();
+  bool IsInitialized() const;
+
+  int ByteSize() const;
+  bool MergePartialFromCodedStream(
+      ::google::protobuf::io::CodedInputStream* input);
+  void SerializeWithCachedSizes(
+      ::google::protobuf::io::CodedOutputStream* output) const;
+  ::google::protobuf::uint8* SerializeWithCachedSizesToArray(::google::protobuf::uint8* output) const;
+  int GetCachedSize() const { return _cached_size_; }
+  private:
+  void SharedCtor();
+  void SharedDtor();
+  void SetCachedSize(int size) const;
+  public:
+
+  ::google::protobuf::Metadata GetMetadata() const;
+
+  // nested types ----------------------------------------------------
+
+  // accessors -------------------------------------------------------
+
+  // optional string sql_query = 1;
+  inline bool has_sql_query() const;
+  inline void clear_sql_query();
+  static const int kSqlQueryFieldNumber = 1;
+  inline const ::std::string& sql_query() const;
+  inline void set_sql_query(const ::std::string& value);
+  inline void set_sql_query(const char* value);
+  inline void set_sql_query(const char* value, size_t size);
+  inline ::std::string* mutable_sql_query();
+  inline ::std::string* release_sql_query();
+  inline void set_allocated_sql_query(::std::string* sql_query);
+
+  // @@protoc_insertion_point(class_scope:exec.bit.ServerPreparedStatementState)
+ private:
+  inline void set_has_sql_query();
+  inline void clear_has_sql_query();
+
+  ::google::protobuf::UnknownFieldSet _unknown_fields_;
+
+  ::std::string* sql_query_;
+
+  mutable int _cached_size_;
+  ::google::protobuf::uint32 _has_bits_[(1 + 31) / 32];
+
+  friend void  protobuf_AddDesc_ExecutionProtos_2eproto();
+  friend void protobuf_AssignDesc_ExecutionProtos_2eproto();
+  friend void protobuf_ShutdownFile_ExecutionProtos_2eproto();
+
+  void InitAsDefaultInstance();
+  static ServerPreparedStatementState* default_instance_;
+};
 // ===================================================================
 
 
@@ -231,6 +331,118 @@ inline void FragmentHandle::set_minor_fragment_id(::google::protobuf::int32 valu
   minor_fragment_id_ = value;
 }
 
+// optional .exec.shared.QueryId parent_query_id = 4;
+inline bool FragmentHandle::has_parent_query_id() const {
+  return (_has_bits_[0] & 0x00000008u) != 0;
+}
+inline void FragmentHandle::set_has_parent_query_id() {
+  _has_bits_[0] |= 0x00000008u;
+}
+inline void FragmentHandle::clear_has_parent_query_id() {
+  _has_bits_[0] &= ~0x00000008u;
+}
+inline void FragmentHandle::clear_parent_query_id() {
+  if (parent_query_id_ != NULL) parent_query_id_->::exec::shared::QueryId::Clear();
+  clear_has_parent_query_id();
+}
+inline const ::exec::shared::QueryId& FragmentHandle::parent_query_id() const {
+  return parent_query_id_ != NULL ? *parent_query_id_ : *default_instance_->parent_query_id_;
+}
+inline ::exec::shared::QueryId* FragmentHandle::mutable_parent_query_id() {
+  set_has_parent_query_id();
+  if (parent_query_id_ == NULL) parent_query_id_ = new ::exec::shared::QueryId;
+  return parent_query_id_;
+}
+inline ::exec::shared::QueryId* FragmentHandle::release_parent_query_id() {
+  clear_has_parent_query_id();
+  ::exec::shared::QueryId* temp = parent_query_id_;
+  parent_query_id_ = NULL;
+  return temp;
+}
+inline void FragmentHandle::set_allocated_parent_query_id(::exec::shared::QueryId* parent_query_id) {
+  delete parent_query_id_;
+  parent_query_id_ = parent_query_id;
+  if (parent_query_id) {
+    set_has_parent_query_id();
+  } else {
+    clear_has_parent_query_id();
+  }
+}
+
+// -------------------------------------------------------------------
+
+// ServerPreparedStatementState
+
+// optional string sql_query = 1;
+inline bool ServerPreparedStatementState::has_sql_query() const {
+  return (_has_bits_[0] & 0x00000001u) != 0;
+}
+inline void ServerPreparedStatementState::set_has_sql_query() {
+  _has_bits_[0] |= 0x00000001u;
+}
+inline void ServerPreparedStatementState::clear_has_sql_query() {
+  _has_bits_[0] &= ~0x00000001u;
+}
+inline void ServerPreparedStatementState::clear_sql_query() {
+  if (sql_query_ != &::google::protobuf::internal::kEmptyString) {
+    sql_query_->clear();
+  }
+  clear_has_sql_query();
+}
+inline const ::std::string& ServerPreparedStatementState::sql_query() const {
+  return *sql_query_;
+}
+inline void ServerPreparedStatementState::set_sql_query(const ::std::string& value) {
+  set_has_sql_query();
+  if (sql_query_ == &::google::protobuf::internal::kEmptyString) {
+    sql_query_ = new ::std::string;
+  }
+  sql_query_->assign(value);
+}
+inline void ServerPreparedStatementState::set_sql_query(const char* value) {
+  set_has_sql_query();
+  if (sql_query_ == &::google::protobuf::internal::kEmptyString) {
+    sql_query_ = new ::std::string;
+  }
+  sql_query_->assign(value);
+}
+inline void ServerPreparedStatementState::set_sql_query(const char* value, size_t size) {
+  set_has_sql_query();
+  if (sql_query_ == &::google::protobuf::internal::kEmptyString) {
+    sql_query_ = new ::std::string;
+  }
+  sql_query_->assign(reinterpret_cast<const char*>(value), size);
+}
+inline ::std::string* ServerPreparedStatementState::mutable_sql_query() {
+  set_has_sql_query();
+  if (sql_query_ == &::google::protobuf::internal::kEmptyString) {
+    sql_query_ = new ::std::string;
+  }
+  return sql_query_;
+}
+inline ::std::string* ServerPreparedStatementState::release_sql_query() {
+  clear_has_sql_query();
+  if (sql_query_ == &::google::protobuf::internal::kEmptyString) {
+    return NULL;
+  } else {
+    ::std::string* temp = sql_query_;
+    sql_query_ = const_cast< ::std::string*>(&::google::protobuf::internal::kEmptyString);
+    return temp;
+  }
+}
+inline void ServerPreparedStatementState::set_allocated_sql_query(::std::string* sql_query) {
+  if (sql_query_ != &::google::protobuf::internal::kEmptyString) {
+    delete sql_query_;
+  }
+  if (sql_query) {
+    set_has_sql_query();
+    sql_query_ = sql_query;
+  } else {
+    clear_has_sql_query();
+    sql_query_ = const_cast< ::std::string*>(&::google::protobuf::internal::kEmptyString);
+  }
+}
+
 
 // @@protoc_insertion_point(namespace_scope)
 


[14/15] drill git commit: DRILL-4420: C++ API for metadata access and prepared statements

Posted by pa...@apache.org.
DRILL-4420: C++ API for metadata access and prepared statements

Add support to the C++ client for metadata querying and prepared
statement requests.

Part of the metadata API, add methods to query for server capabilities.
As of now, this interface is not backed up by any RPC exchange so
the information is pretty much static, and match Drill 1.8.0
current capabilities.


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

Branch: refs/heads/master
Commit: 166c4ce7600b5571249a6748dd57383479313e2e
Parents: 3a35a42
Author: Laurent Goujon <la...@dremio.com>
Authored: Mon Aug 15 14:17:35 2016 -0700
Committer: Parth Chandra <pa...@apache.org>
Committed: Tue Nov 1 11:29:58 2016 -0700

----------------------------------------------------------------------
 contrib/native/client/CMakeLists.txt            |    9 +-
 .../native/client/example/querySubmitter.cpp    |   24 +-
 .../native/client/src/clientlib/CMakeLists.txt  |    8 +-
 .../client/src/clientlib/collectionsImpl.hpp    |  123 ++
 .../native/client/src/clientlib/drillClient.cpp |  237 ++--
 .../client/src/clientlib/drillClientImpl.cpp    | 1315 +++++++++++-------
 .../client/src/clientlib/drillClientImpl.hpp    |  392 ++++--
 contrib/native/client/src/clientlib/env.h.in    |    9 +
 contrib/native/client/src/clientlib/errmsgs.cpp |    2 +-
 .../native/client/src/clientlib/fieldmeta.cpp   |  406 ++++++
 .../native/client/src/clientlib/metadata.cpp    |  748 ++++++++++
 .../native/client/src/clientlib/metadata.hpp    |  288 ++++
 .../native/client/src/clientlib/recordBatch.cpp |   12 +-
 .../native/client/src/clientlib/rpcDecoder.cpp  |  153 --
 .../native/client/src/clientlib/rpcDecoder.hpp  |   38 -
 .../native/client/src/clientlib/rpcEncoder.cpp  |  109 --
 .../native/client/src/clientlib/rpcEncoder.hpp  |   55 -
 .../native/client/src/clientlib/rpcMessage.cpp  |  241 ++++
 .../native/client/src/clientlib/rpcMessage.hpp  |   15 +-
 contrib/native/client/src/clientlib/utils.cpp   |    7 +
 contrib/native/client/src/clientlib/utils.hpp   |    1 -
 .../native/client/src/clientlib/y2038/time64.c  |   18 +-
 .../client/src/clientlib/zookeeperClient.cpp    |  168 +++
 .../client/src/clientlib/zookeeperClient.hpp    |   71 +
 .../client/src/include/drill/collections.hpp    |  179 +++
 .../native/client/src/include/drill/common.hpp  |   18 +
 .../client/src/include/drill/drillClient.hpp    |  998 ++++++++++++-
 .../native/client/src/include/drill/drillc.hpp  |    2 +
 .../client/src/include/drill/fieldmeta.hpp      |  122 ++
 .../src/include/drill/preparedStatement.hpp     |   38 +
 .../client/src/include/drill/recordBatch.hpp    |   35 +-
 contrib/native/client/src/test/CMakeLists.txt   |    1 +
 .../native/client/src/test/CollectionsTest.cpp  |  215 +++
 protocol/src/main/protobuf/User.proto           |    2 +-
 34 files changed, 4819 insertions(+), 1240 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/166c4ce7/contrib/native/client/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/contrib/native/client/CMakeLists.txt b/contrib/native/client/CMakeLists.txt
index e61eb9c..7e22ce8 100644
--- a/contrib/native/client/CMakeLists.txt
+++ b/contrib/native/client/CMakeLists.txt
@@ -57,10 +57,9 @@ project(drillclient
         )
 
 message("Project Dir = ${PROJECT_SOURCE_DIR}")
-message("Project version = ${PROJECT_VERSION} ")
+message("Project Version = ${PROJECT_VERSION} ")
 message("Source Dir = ${CMAKE_SOURCE_DIR} ")
 
-cmake_policy(SET CMP0043 NEW)
 
 set(CMAKE_MODULE_PATH ${CMAKE_MODULE_PATH} "${CMAKE_SOURCE_DIR}/cmakeModules/")
 
@@ -71,6 +70,12 @@ execute_process(
     OUTPUT_VARIABLE GIT_COMMIT_PROP
     OUTPUT_STRIP_TRAILING_WHITESPACE
     )
+execute_process(
+    COMMAND git log -1 --format="%H"
+    WORKING_DIRECTORY ${CMAKE_SOURCE_DIR}
+    OUTPUT_VARIABLE GIT_SHA_PROP
+    OUTPUT_STRIP_TRAILING_WHITESPACE
+    )
 STRING(REPLACE . " " GIT_COMMIT_PROP "${GIT_COMMIT_PROP}")
 STRING(REPLACE \" "" GIT_COMMIT_PROP "${GIT_COMMIT_PROP}")
 set(GIT_COMMIT_PROP "\"${GIT_COMMIT_PROP}\"")

http://git-wip-us.apache.org/repos/asf/drill/blob/166c4ce7/contrib/native/client/example/querySubmitter.cpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/example/querySubmitter.cpp b/contrib/native/client/example/querySubmitter.cpp
index d507d1b..306db56 100644
--- a/contrib/native/client/example/querySubmitter.cpp
+++ b/contrib/native/client/example/querySubmitter.cpp
@@ -316,8 +316,8 @@ int main(int argc, char* argv[]) {
         std::vector<Drill::RecordIterator*> recordIterators;
         std::vector<Drill::RecordIterator*>::iterator recordIterIter;
 
-        std::vector<Drill::QueryHandle_t*> queryHandles;
-        std::vector<Drill::QueryHandle_t*>::iterator queryHandleIter;
+        std::vector<Drill::QueryHandle_t> queryHandles;
+        std::vector<Drill::QueryHandle_t>::iterator queryHandleIter;
 
         Drill::DrillClient client;
 #if defined _WIN32 || defined _WIN64
@@ -327,7 +327,7 @@ int main(int argc, char* argv[]) {
 		strcpy(logpathPrefix,tempPath);
 		strcat(logpathPrefix, "\\drillclient");
 #else
-		char* logpathPrefix = "/var/log/drill/drillclient";
+		const char* logpathPrefix = "/var/log/drill/drillclient";
 #endif
 		// To log to file
         Drill::DrillClient::initLogging(logpathPrefix, l);
@@ -411,27 +411,25 @@ int main(int argc, char* argv[]) {
         }else{
             if(bSyncSend){
                 for(queryInpIter = queryInputs.begin(); queryInpIter != queryInputs.end(); queryInpIter++) {
-                    Drill::QueryHandle_t* qryHandle = new Drill::QueryHandle_t;
-                    client.submitQuery(type, *queryInpIter, QueryResultsListener, NULL, qryHandle);
-                    client.registerSchemaChangeListener(qryHandle, SchemaListener);
+                    Drill::QueryHandle_t qryHandle;
+                    client.submitQuery(type, *queryInpIter, QueryResultsListener, NULL, &qryHandle);
+                    client.registerSchemaChangeListener(&qryHandle, SchemaListener);
                     
                     client.waitForResults();
 
-                    client.freeQueryResources(qryHandle);
-                    delete qryHandle;
+                    client.freeQueryResources(&qryHandle);
                 }
 
             }else{
                 for(queryInpIter = queryInputs.begin(); queryInpIter != queryInputs.end(); queryInpIter++) {
-                    Drill::QueryHandle_t* qryHandle = new Drill::QueryHandle_t;
-                    client.submitQuery(type, *queryInpIter, QueryResultsListener, NULL, qryHandle);
-                    client.registerSchemaChangeListener(qryHandle, SchemaListener);
+                    Drill::QueryHandle_t qryHandle;
+                    client.submitQuery(type, *queryInpIter, QueryResultsListener, NULL, &qryHandle);
+                    client.registerSchemaChangeListener(&qryHandle, SchemaListener);
                     queryHandles.push_back(qryHandle);
                 }
                 client.waitForResults();
                 for(queryHandleIter = queryHandles.begin(); queryHandleIter != queryHandles.end(); queryHandleIter++) {
-                    client.freeQueryResources(*queryHandleIter);
-                    delete *queryHandleIter;
+                    client.freeQueryResources(&*queryHandleIter);
                 }
             }
         }

http://git-wip-us.apache.org/repos/asf/drill/blob/166c4ce7/contrib/native/client/src/clientlib/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/clientlib/CMakeLists.txt b/contrib/native/client/src/clientlib/CMakeLists.txt
index a2e7052..68326e2 100644
--- a/contrib/native/client/src/clientlib/CMakeLists.txt
+++ b/contrib/native/client/src/clientlib/CMakeLists.txt
@@ -22,12 +22,14 @@ set (CLIENTLIB_SRC_FILES
     ${CMAKE_CURRENT_SOURCE_DIR}/decimalUtils.cpp
     ${CMAKE_CURRENT_SOURCE_DIR}/drillClient.cpp
     ${CMAKE_CURRENT_SOURCE_DIR}/drillClientImpl.cpp
+    ${CMAKE_CURRENT_SOURCE_DIR}/fieldmeta.cpp
+    ${CMAKE_CURRENT_SOURCE_DIR}/metadata.cpp
     ${CMAKE_CURRENT_SOURCE_DIR}/recordBatch.cpp
-    ${CMAKE_CURRENT_SOURCE_DIR}/rpcEncoder.cpp
-    ${CMAKE_CURRENT_SOURCE_DIR}/rpcDecoder.cpp
+    ${CMAKE_CURRENT_SOURCE_DIR}/rpcMessage.cpp
     ${CMAKE_CURRENT_SOURCE_DIR}/errmsgs.cpp
     ${CMAKE_CURRENT_SOURCE_DIR}/logger.cpp
     ${CMAKE_CURRENT_SOURCE_DIR}/utils.cpp
+    ${CMAKE_CURRENT_SOURCE_DIR}/zookeeperClient.cpp
     )
 
 include_directories(${CMAKE_CURRENT_SOURCE_DIR} ${CMAKE_CURRENT_SOURCE_DIR}/../include )
@@ -43,7 +45,7 @@ set_property(
 
 if(MSVC)
     set(CMAKE_CXX_FLAGS "/EHsc")
-    add_definitions(-DDRILL_CLIENT_EXPORTS)
+    add_definitions(-DDRILL_CLIENT_EXPORTS -D_SCL_SECURE_NO_WARNINGS)
 endif()
 
 add_library(drillClient SHARED ${CLIENTLIB_SRC_FILES} )

http://git-wip-us.apache.org/repos/asf/drill/blob/166c4ce7/contrib/native/client/src/clientlib/collectionsImpl.hpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/clientlib/collectionsImpl.hpp b/contrib/native/client/src/clientlib/collectionsImpl.hpp
new file mode 100644
index 0000000..be1b54f
--- /dev/null
+++ b/contrib/native/client/src/clientlib/collectionsImpl.hpp
@@ -0,0 +1,123 @@
+/*
+ * 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.
+ */
+
+#ifndef DRILL_COLLECTIONSIMPL_H
+#define DRILL_COLLECTIONSIMPL_H
+
+#include <iterator>
+#include <drill/collections.hpp>
+
+namespace Drill {
+namespace impl {
+template<typename T, typename Iterator>
+class DrillContainerIterator: public DrillIteratorImpl<T> {
+public:
+	typedef DrillContainerIterator<T, Iterator> type;
+	typedef DrillIteratorImpl<T> supertype;
+	typedef typename supertype::iterator iterator;
+	typedef typename iterator::value_type value_type;
+	typedef typename iterator::reference reference;
+	typedef typename iterator::pointer pointer;
+
+	DrillContainerIterator(Iterator it): supertype(), m_it(it) {};
+
+	operator typename DrillIteratorImpl<const T>::iterator_ptr() const { return typename DrillIteratorImpl<const T>::iterator_ptr(new DrillContainerIterator<const T, Iterator>(m_it)); }
+
+	reference operator*() const { return m_it.operator *();}
+	pointer   operator->() const { return m_it.operator->(); }
+
+	iterator& operator++() { m_it++; return *this; }
+
+	bool operator==(const iterator& x) const {
+		const type& other(dynamic_cast<const type&>(x));
+		return m_it == other.m_it;
+	}
+
+	bool operator!=(const iterator& x) const { return !(*this==x); }
+
+private:
+	Iterator m_it;
+};
+
+template<typename T, typename Container>
+class DrillContainerCollection: public DrillCollectionImpl<T> {
+public:
+	typedef DrillCollectionImpl<T> supertype;
+	typedef typename supertype::value_type value_type;
+	typedef typename supertype::iterator iterator;
+	typedef typename supertype::const_iterator const_iterator;
+
+	typedef typename supertype::iterator_ptr iterator_ptr;
+	typedef typename supertype::const_iterator_ptr const_iterator_ptr;
+
+	DrillContainerCollection(): supertype(), m_container() {};
+
+	Container& operator*() { return &m_container; }
+	const Container& operator*() const { return &m_container; }
+	Container* operator->() { return &m_container; }
+	const Container* operator->() const { return &m_container; }
+
+	iterator_ptr begin() { return iterator_ptr(new IteratorImpl(m_container.begin())); }
+	const_iterator_ptr begin() const { return const_iterator_ptr(new ConstIteratorImpl(m_container.begin())); }
+	iterator_ptr end() { return iterator_ptr(new IteratorImpl(m_container.end())); }
+	const_iterator_ptr end() const { return const_iterator_ptr(new ConstIteratorImpl(m_container.end())); }
+
+private:
+	typedef DrillContainerIterator<value_type, typename Container::iterator> IteratorImpl;
+	typedef DrillContainerIterator<const value_type, typename Container::const_iterator> ConstIteratorImpl;
+
+	Container m_container;
+};
+} /* namespace impl */
+
+
+/**
+ * Drill collection backed up by a vector
+ * Offer a view over a collection of Iface instances,
+ * where concrete implementation of Iface is T
+ */
+template<typename Iface, typename T>
+class DrillVector: public DrillCollection<Iface> {
+public:
+	DrillVector(): DrillCollection<Iface>(ImplPtr(new Impl())) {};
+
+	void clear() {
+		Impl& impl = static_cast<Impl&>(**this);
+		impl->clear();
+	}
+
+	void push_back( const T& value ) {
+		Impl& impl = static_cast<Impl&>(**this);
+		impl->push_back(value);
+	}
+
+	void reserve(std::size_t new_cap) {
+		Impl& impl = static_cast<Impl&>(**this);
+		impl->reserve(new_cap);
+	}
+
+
+private:
+	typedef impl::DrillContainerCollection<Iface, std::vector<T> > Impl;
+	typedef boost::shared_ptr<Impl> ImplPtr;
+};
+}
+
+
+
+#endif /* DRILL_COLLECTIONSIMPL_H */

http://git-wip-us.apache.org/repos/asf/drill/blob/166c4ce7/contrib/native/client/src/clientlib/drillClient.cpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/clientlib/drillClient.cpp b/contrib/native/client/src/clientlib/drillClient.cpp
index 1251058..20a466e 100644
--- a/contrib/native/client/src/clientlib/drillClient.cpp
+++ b/contrib/native/client/src/clientlib/drillClient.cpp
@@ -20,11 +20,11 @@
 #include <boost/assign.hpp>
 #include "drill/common.hpp"
 #include "drill/drillClient.hpp"
+#include "drill/fieldmeta.hpp"
 #include "drill/recordBatch.hpp"
 #include "drillClientImpl.hpp"
 #include "errmsgs.hpp"
 #include "logger.hpp"
-
 #include "Types.pb.h"
 
 namespace Drill{
@@ -173,83 +173,86 @@ FieldDefPtr RecordIterator::getColDefs(){
     if(m_pQueryResult->hasError()){
         return DrillClientQueryResult::s_emptyColDefs;
     }
+
+    if (this->m_pColDefs != NULL && !this->hasSchemaChanged()) {
+    	return this->m_pColDefs;
+    }
+
     //NOTE: if query is cancelled, return whatever you have. Client applications job to deal with it.
-    if(this->m_pColDefs==NULL || this->hasSchemaChanged()){
-        if(this->m_pCurrentRecordBatch==NULL){
-            this->m_pQueryResult->waitForData();
-            if(m_pQueryResult->hasError()){
-                return DrillClientQueryResult::s_emptyColDefs;
-            }
-        }
-        if(this->hasSchemaChanged()){
-            if(m_pColDefs!=NULL){
-                for(std::vector<Drill::FieldMetadata*>::iterator it=m_pColDefs->begin();
-                        it!=m_pColDefs->end();
-                        ++it){
-                    delete *it;
-                }
-                m_pColDefs->clear();
-                //delete m_pColDefs; m_pColDefs=NULL;
-            }
-        }
-        FieldDefPtr pColDefs(  new std::vector<Drill::FieldMetadata*>);
-        {   //lock after we come out of the  wait.
-            boost::lock_guard<boost::mutex> bufferLock(this->m_recordBatchMutex);
-            boost::shared_ptr< std::vector<Drill::FieldMetadata*> >  currentColDefs=DrillClientQueryResult::s_emptyColDefs;
-            if(this->m_pCurrentRecordBatch!=NULL){
-                currentColDefs=this->m_pCurrentRecordBatch->getColumnDefs();
-            }else{
-                // This is reached only when the first results have been received but
-                // the getNext call has not been made to retrieve the record batch
-                RecordBatch* pR=this->m_pQueryResult->peekNext();
-                if(pR!=NULL){
-                    currentColDefs=pR->getColumnDefs();
-                }
-            }
-            for(std::vector<Drill::FieldMetadata*>::iterator it=currentColDefs->begin(); it!=currentColDefs->end(); ++it){
-                Drill::FieldMetadata* fmd= new Drill::FieldMetadata;
-                fmd->copy(*(*it));//Yup, that's 2 stars
-                pColDefs->push_back(fmd);
-            }
-        }
-        this->m_pColDefs = pColDefs;
+    if(this->m_pCurrentRecordBatch==NULL){
+    	this->m_pQueryResult->waitForData();
+    	if(m_pQueryResult->hasError()){
+    		return DrillClientQueryResult::s_emptyColDefs;
+    	}
+    }
+    if(this->hasSchemaChanged()){
+    	if(m_pColDefs!=NULL){
+    		for(std::vector<Drill::FieldMetadata*>::iterator it=m_pColDefs->begin();
+    				it!=m_pColDefs->end();
+    				++it){
+    			delete *it;
+    		}
+    		m_pColDefs->clear();
+    		//delete m_pColDefs; m_pColDefs=NULL;
+    	}
+    }
+    FieldDefPtr pColDefs(  new std::vector<Drill::FieldMetadata*>);
+    {   //lock after we come out of the  wait.
+    	boost::lock_guard<boost::mutex> bufferLock(this->m_recordBatchMutex);
+    	boost::shared_ptr< std::vector<Drill::FieldMetadata*> >  currentColDefs=DrillClientQueryResult::s_emptyColDefs;
+    	if(this->m_pCurrentRecordBatch!=NULL){
+    		currentColDefs=this->m_pCurrentRecordBatch->getColumnDefs();
+    	}else{
+    		// This is reached only when the first results have been received but
+    		// the getNext call has not been made to retrieve the record batch
+    		RecordBatch* pR=this->m_pQueryResult->peekNext();
+    		if(pR!=NULL){
+    			currentColDefs=pR->getColumnDefs();
+    		}
+    	}
+    	for(std::vector<Drill::FieldMetadata*>::const_iterator it=currentColDefs->begin(); it!=currentColDefs->end(); ++it){
+    		Drill::FieldMetadata* fmd= new Drill::FieldMetadata;
+    		fmd->copy(*(*it));//Yup, that's 2 stars
+    		pColDefs->push_back(fmd);
+    	}
     }
+    this->m_pColDefs = pColDefs;
     return this->m_pColDefs;
 }
 
 status_t RecordIterator::next(){
     status_t ret=QRY_SUCCESS;
     this->m_currentRecord++;
+    if(this->m_pQueryResult->isCancelled()){
+    	return QRY_CANCEL;
+    }
 
-    if(!this->m_pQueryResult->isCancelled()){
-        if(this->m_pCurrentRecordBatch==NULL || this->m_currentRecord==this->m_pCurrentRecordBatch->getNumRecords()){
-            boost::lock_guard<boost::mutex> bufferLock(this->m_recordBatchMutex);
-            if(this->m_pCurrentRecordBatch !=NULL){
-                DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Deleted old Record batch " << (void*) m_pCurrentRecordBatch << std::endl;)
-                delete this->m_pCurrentRecordBatch; //free the previous record batch
-                this->m_pCurrentRecordBatch=NULL;
-            }
-            this->m_currentRecord=0;
-            this->m_pQueryResult->waitForData();
-            if(m_pQueryResult->hasError()){
-                return m_pQueryResult->getErrorStatus();
-            }
-            this->m_pCurrentRecordBatch=this->m_pQueryResult->getNext();
-            if(this->m_pCurrentRecordBatch != NULL){
-                DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Fetched new Record batch " << std::endl;)
-            }else{
-                DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "No new Record batch found " << std::endl;)
-            }
-            if(this->m_pCurrentRecordBatch==NULL || this->m_pCurrentRecordBatch->getNumRecords()==0){
-                DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "No more data." << std::endl;)
-                ret = QRY_NO_MORE_DATA;
-            }else if(this->m_pCurrentRecordBatch->hasSchemaChanged()){
-                ret=QRY_SUCCESS_WITH_INFO;
-            }
-        }
-    }else{
-        ret=QRY_CANCEL;
+    if(this->m_pCurrentRecordBatch==NULL || this->m_currentRecord==this->m_pCurrentRecordBatch->getNumRecords()){
+    	boost::lock_guard<boost::mutex> bufferLock(this->m_recordBatchMutex);
+    	if(this->m_pCurrentRecordBatch !=NULL){
+    		DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Deleted old Record batch " << (void*) m_pCurrentRecordBatch << std::endl;)
+                		delete this->m_pCurrentRecordBatch; //free the previous record batch
+    		this->m_pCurrentRecordBatch=NULL;
+    	}
+    	this->m_currentRecord=0;
+    	this->m_pQueryResult->waitForData();
+    	if(m_pQueryResult->hasError()){
+    		return m_pQueryResult->getErrorStatus();
+    	}
+    	this->m_pCurrentRecordBatch=this->m_pQueryResult->getNext();
+    	if(this->m_pCurrentRecordBatch != NULL){
+    		DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Fetched new Record batch " << std::endl;)
+    	}else{
+    		DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "No new Record batch found " << std::endl;)
+    	}
+    	if(this->m_pCurrentRecordBatch==NULL || this->m_pCurrentRecordBatch->getNumRecords()==0){
+    		DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "No more data." << std::endl;)
+                		ret = QRY_NO_MORE_DATA;
+    	}else if(this->m_pCurrentRecordBatch->hasSchemaChanged()){
+    		ret=QRY_SUCCESS_WITH_INFO;
+    	}
     }
+
     return ret;
 }
 
@@ -258,30 +261,28 @@ status_t RecordIterator::getCol(size_t i, void** b, size_t* sz){
     //TODO: check fields out of bounds without calling getColDefs
     //if(i>=getColDefs().size()) return QRY_OUT_OF_BOUNDS;
     //return raw byte buffer
-    if(!this->m_pQueryResult->isCancelled()){
-        const ValueVectorBase* pVector=this->m_pCurrentRecordBatch->getFields()[i]->getVector();
-        if(!pVector->isNull(this->m_currentRecord)){
-            *b=pVector->getRaw(this->m_currentRecord);
-            *sz=pVector->getSize(this->m_currentRecord);
-        }else{
-            *b=NULL;
-            *sz=0;
-
-        }
-        return QRY_SUCCESS;
+    if(this->m_pQueryResult->isCancelled()){
+    	return QRY_CANCEL;
+    }
+    const ValueVectorBase* pVector=this->m_pCurrentRecordBatch->getFields()[i]->getVector();
+    if(!pVector->isNull(this->m_currentRecord)){
+    	*b=pVector->getRaw(this->m_currentRecord);
+    	*sz=pVector->getSize(this->m_currentRecord);
     }else{
-        return QRY_CANCEL;
+    	*b=NULL;
+    	*sz=0;
     }
+    return QRY_SUCCESS;
 }
 
 /* true if ith column in the current record is NULL. */
 bool RecordIterator::isNull(size_t i){
-    if(!this->m_pQueryResult->isCancelled()){
-        const ValueVectorBase* pVector=this->m_pCurrentRecordBatch->getFields()[i]->getVector();
-        return pVector->isNull(this->m_currentRecord);
-    }else{
-        return false;
+    if(this->m_pQueryResult->isCancelled()){
+    	return false;
     }
+
+    const ValueVectorBase* pVector=this->m_pCurrentRecordBatch->getFields()[i]->getVector();
+    return pVector->isNull(this->m_currentRecord);
 }
 
 status_t RecordIterator::cancel(){
@@ -329,19 +330,15 @@ DrillClient::~DrillClient(){
 }
 
 connectionStatus_t DrillClient::connect(const char* connectStr, const char* defaultSchema){
-    connectionStatus_t ret=CONN_SUCCESS;
-    ret=this->m_pImpl->connect(connectStr);
     DrillUserProperties props;
     std::string schema(defaultSchema);
     props.setProperty(USERPROP_SCHEMA,  schema);
-    if(ret==CONN_SUCCESS){
-        if(defaultSchema!=NULL){
-            ret=this->m_pImpl->validateHandshake(&props);
-        }else{
-            ret=this->m_pImpl->validateHandshake(NULL);
-        }
+    if (defaultSchema != NULL) {
+    	return connect(connectStr, static_cast<DrillUserProperties*>(NULL));
+    }
+    else {
+    	return connect(connectStr, &props);
     }
-    return ret;
 }
 
 connectionStatus_t DrillClient::connect(const char* connectStr, DrillUserProperties* properties){
@@ -366,14 +363,12 @@ void DrillClient::close() {
 }
 
 status_t DrillClient::submitQuery(Drill::QueryType t, const std::string& plan, pfnQueryResultsListener listener, void* listenerCtx, QueryHandle_t* qHandle){
-
     ::exec::shared::QueryType castedType = static_cast< ::exec::shared::QueryType> (t);
     DrillClientQueryResult* pResult=this->m_pImpl->SubmitQuery(castedType, plan, listener, listenerCtx);
+    *qHandle=static_cast<QueryHandle_t>(pResult);
     if(pResult==NULL){
-        *qHandle=NULL;
         return (status_t)this->m_pImpl->getError()->status;
     }
-    *qHandle=(QueryHandle_t)pResult;
     return QRY_SUCCESS;
 }
 
@@ -387,14 +382,32 @@ RecordIterator* DrillClient::submitQuery(Drill::QueryType t, const std::string&
     return pIter;
 }
 
+status_t DrillClient::prepareQuery(const std::string& sql, pfnPreparedStatementListener listener, void* listenerCtx, QueryHandle_t* qHandle) {
+	DrillClientPrepareHandle* pResult=this->m_pImpl->PrepareQuery(sql, listener, listenerCtx);
+	*qHandle=static_cast<QueryHandle_t>(pResult);
+	if(pResult==NULL){
+		return static_cast<status_t>(this->m_pImpl->getError()->status);
+	}
+	return QRY_SUCCESS;
+}
+
+status_t DrillClient::executeQuery(const PreparedStatement& pstmt, pfnQueryResultsListener listener, void* listenerCtx, QueryHandle_t* qHandle) {
+	DrillClientQueryResult* pResult=this->m_pImpl->ExecuteQuery(pstmt, listener, listenerCtx);
+	*qHandle=static_cast<QueryHandle_t>(pResult);
+	if(pResult==NULL){
+		return static_cast<status_t>(this->m_pImpl->getError()->status);
+	}
+	return QRY_SUCCESS;
+}
+
 void* DrillClient::getApplicationContext(QueryHandle_t handle){
     assert(handle!=NULL);
-    return ((DrillClientQueryResult*)handle)->getListenerContext();
+    return (static_cast<DrillClientQueryHandle*>(handle))->getApplicationContext();
 }
 
 status_t DrillClient::getQueryStatus(QueryHandle_t handle){
     assert(handle!=NULL);
-    return ((DrillClientQueryResult*)handle)->getQueryStatus();
+    return static_cast<DrillClientQueryHandle*>(handle)->getQueryStatus();
 }
 
 std::string& DrillClient::getError(){
@@ -402,7 +415,7 @@ std::string& DrillClient::getError(){
 }
 
 const std::string& DrillClient::getError(QueryHandle_t handle){
-    return ((DrillClientQueryResult*)handle)->getError()->msg;
+    return static_cast<DrillClientQueryHandle*>(handle)->getError()->msg;
 }
 
 void DrillClient::waitForResults(){
@@ -410,13 +423,23 @@ void DrillClient::waitForResults(){
 }
 
 void DrillClient::registerSchemaChangeListener(QueryHandle_t* handle, pfnSchemaListener l){
-    if(handle!=NULL){
-        ((DrillClientQueryResult*)(*handle))->registerSchemaChangeListener(l);
+	if (!handle) {
+		return;
+	}
+
+	// Let's ensure that handle is really an instance of DrillClientQueryResult
+	// by using dynamic_cast to verify. Since void is not a class, we first have
+	// to static_cast to a DrillClientQueryHandle
+	DrillClientQueryHandle* pHandle = static_cast<DrillClientQueryHandle*>(*handle);
+	DrillClientQueryResult* result = dynamic_cast<DrillClientQueryResult*>(pHandle);
+
+	if (result) {
+        result->registerSchemaChangeListener(l);
     }
 }
 
 void DrillClient::freeQueryResources(QueryHandle_t* handle){
-    delete (DrillClientQueryResult*)(*handle);
+	this->m_pImpl->freeQueryResources(static_cast<DrillClientQueryHandle*>(*handle));
     *handle=NULL;
 }
 
@@ -424,4 +447,12 @@ void DrillClient::freeRecordBatch(RecordBatch* pRecordBatch){
     delete pRecordBatch;
 }
 
+Metadata* DrillClient::getMetadata() {
+    return this->m_pImpl->getMetadata();
+}
+
+void DrillClient::freeMetadata(Metadata** metadata) {
+    this->m_pImpl->freeMetadata(static_cast<meta::DrillMetadata*>(*metadata));
+    *metadata = NULL;
+}
 } // namespace Drill


[13/15] drill git commit: DRILL-4420: C++ API for metadata access and prepared statements

Posted by pa...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/166c4ce7/contrib/native/client/src/clientlib/drillClientImpl.cpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/clientlib/drillClientImpl.cpp b/contrib/native/client/src/clientlib/drillClientImpl.cpp
index b5d5a31..7ecf910 100644
--- a/contrib/native/client/src/clientlib/drillClientImpl.cpp
+++ b/contrib/native/client/src/clientlib/drillClientImpl.cpp
@@ -19,32 +19,30 @@
 
 #include "drill/common.hpp"
 #include <queue>
-#include <string.h>
+#include <string>
 #include <boost/asio.hpp>
+#include <boost/assign.hpp>
 #include <boost/bind.hpp>
 #include <boost/date_time/posix_time/posix_time.hpp>
 #include <boost/date_time/posix_time/posix_time_duration.hpp>
+#include <boost/functional/factory.hpp>
 #include <boost/lexical_cast.hpp>
 #include <boost/thread.hpp>
-#ifdef _WIN32
-#include <zookeeper.h>
-#else
-#include <zookeeper/zookeeper.h>
-#endif
-#include <boost/assign.hpp>
+
 
 #include "drill/drillClient.hpp"
+#include "drill/fieldmeta.hpp"
 #include "drill/recordBatch.hpp"
 #include "drillClientImpl.hpp"
+#include "collectionsImpl.hpp"
 #include "errmsgs.hpp"
 #include "logger.hpp"
-#include "rpcEncoder.hpp"
-#include "rpcDecoder.hpp"
+#include "metadata.hpp"
 #include "rpcMessage.hpp"
 #include "utils.hpp"
-
 #include "GeneralRPC.pb.h"
 #include "UserBitShared.pb.h"
+#include "zookeeperClient.hpp"
 
 namespace Drill{
 
@@ -56,70 +54,57 @@ static std::map<exec::shared::QueryResult_QueryState, status_t> QUERYSTATE_TO_ST
     (exec::shared::QueryResult_QueryState_FAILED, QRY_FAILED)
     ;
 
-RpcEncoder DrillClientImpl::s_encoder;
-RpcDecoder DrillClientImpl::s_decoder;
-
-std::string debugPrintQid(const exec::shared::QueryId& qid){
+static std::string debugPrintQid(const exec::shared::QueryId& qid){
     return std::string("[")+boost::lexical_cast<std::string>(qid.part1()) +std::string(":") + boost::lexical_cast<std::string>(qid.part2())+std::string("] ");
 }
 
-void setSocketTimeout(boost::asio::ip::tcp::socket& socket, int32_t timeout){
-#if defined _WIN32
-    int32_t timeoutMsecs=timeout*1000;
-    setsockopt(socket.native(), SOL_SOCKET, SO_RCVTIMEO, (const char*)&timeoutMsecs, sizeof(timeoutMsecs));
-    setsockopt(socket.native(), SOL_SOCKET, SO_SNDTIMEO, (const char*)&timeoutMsecs, sizeof(timeoutMsecs));
-#else
-    struct timeval tv;
-    tv.tv_sec  = timeout;
-    tv.tv_usec = 0;
-    int e=0;
-    e=setsockopt(socket.native(), SOL_SOCKET, SO_RCVTIMEO, &tv, sizeof(tv));
-    e=setsockopt(socket.native(), SOL_SOCKET, SO_SNDTIMEO, &tv, sizeof(tv));
-#endif
-}
-
 connectionStatus_t DrillClientImpl::connect(const char* connStr){
     std::string pathToDrill, protocol, hostPortStr;
     std::string host;
     std::string port;
-    if(!this->m_bIsConnected){
-        m_connectStr=connStr;
-        Utils::parseConnectStr(connStr, pathToDrill, protocol, hostPortStr);
-        if(!strcmp(protocol.c_str(), "zk")){
-            ZookeeperImpl zook;
-            std::vector<std::string> drillbits;
-            int err = zook.getAllDrillbits(hostPortStr.c_str(), pathToDrill.c_str(), drillbits);
+
+    if (this->m_bIsConnected) {
+        if(std::strcmp(connStr, m_connectStr.c_str())){ // trying to connect to a different address is not allowed if already connected
+            return handleConnError(CONN_ALREADYCONNECTED, getMessage(ERR_CONN_ALREADYCONN));
+        }
+        return CONN_SUCCESS;
+    }
+
+    m_connectStr=connStr;
+    Utils::parseConnectStr(connStr, pathToDrill, protocol, hostPortStr);
+    if(protocol == "zk"){
+        ZookeeperClient zook(pathToDrill);
+        std::vector<std::string> drillbits;
+        int err = zook.getAllDrillbits(hostPortStr, drillbits);
+        if(!err){
+            Utils::shuffle(drillbits);
+            exec::DrillbitEndpoint endpoint;
+            err = zook.getEndPoint(drillbits[drillbits.size() -1], endpoint);// get the last one in the list
             if(!err){
-                Utils::shuffle(drillbits);
-                exec::DrillbitEndpoint endpoint;
-                err = zook.getEndPoint(drillbits, drillbits.size()-1, endpoint);// get the last one in the list
-                if(!err){
-                    host=boost::lexical_cast<std::string>(endpoint.address());
-                    port=boost::lexical_cast<std::string>(endpoint.user_port());
-                }
+                host=boost::lexical_cast<std::string>(endpoint.address());
+                port=boost::lexical_cast<std::string>(endpoint.user_port());
             }
-            if(err){
-                return handleConnError(CONN_ZOOKEEPER_ERROR, getMessage(ERR_CONN_ZOOKEEPER, zook.getError().c_str()));
-            }
-            zook.close();
-            m_bIsDirectConnection=true;  
-        }else if(!strcmp(protocol.c_str(), "local")){
-            boost::lock_guard<boost::mutex> lock(m_dcMutex);//strtok is not reentrant
-            char tempStr[MAX_CONNECT_STR+1];
-            strncpy(tempStr, hostPortStr.c_str(), MAX_CONNECT_STR); tempStr[MAX_CONNECT_STR]=0;
-            host=strtok(tempStr, ":");
-            port=strtok(NULL, "");
-            m_bIsDirectConnection=false;  
-        }else{
-            return handleConnError(CONN_INVALID_INPUT, getMessage(ERR_CONN_UNKPROTO, protocol.c_str()));
+            DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Choosing drillbit <" << (drillbits.size() - 1)  << ">. Selected " << endpoint.DebugString() << std::endl;)
+
         }
-        DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Connecting to endpoint: " << host << ":" << port << std::endl;)
-        connectionStatus_t ret = this->connect(host.c_str(), port.c_str());
-        return ret;
-    }else if(std::strcmp(connStr, m_connectStr.c_str())){ // tring to connect to a different address is not allowed if already connected
-        return handleConnError(CONN_ALREADYCONNECTED, getMessage(ERR_CONN_ALREADYCONN));
+        if(err){
+            return handleConnError(CONN_ZOOKEEPER_ERROR, getMessage(ERR_CONN_ZOOKEEPER, zook.getError().c_str()));
+        }
+        zook.close();
+        m_bIsDirectConnection=true;
+    }else if(protocol == "local"){
+        boost::lock_guard<boost::mutex> lock(m_dcMutex);//strtok is not reentrant
+        char tempStr[MAX_CONNECT_STR+1];
+        strncpy(tempStr, hostPortStr.c_str(), MAX_CONNECT_STR); tempStr[MAX_CONNECT_STR]=0;
+        host=strtok(tempStr, ":");
+        port=strtok(NULL, "");
+        m_bIsDirectConnection=false;
+    }else{
+        return handleConnError(CONN_INVALID_INPUT, getMessage(ERR_CONN_UNKPROTO, protocol.c_str()));
     }
-    return CONN_SUCCESS;
+    DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Connecting to endpoint: " << host << ":" << port << std::endl;)
+    connectionStatus_t ret = this->connect(host.c_str(), port.c_str());
+    return ret;
 }
 
 connectionStatus_t DrillClientImpl::connect(const char* host, const char* port){
@@ -140,7 +125,7 @@ connectionStatus_t DrillClientImpl::connect(const char* host, const char* port){
             return handleConnError(CONN_FAILURE, getMessage(ERR_CONN_FAILURE, host, port, ec.message().c_str()));
         }
 
-    }catch(std::exception e){
+    }catch(const std::exception & e){
         // Handle case when the hostname cannot be resolved. "resolve" is hard-coded in boost asio resolver.resolve
         if (!strcmp(e.what(), "resolve")) {
             return handleConnError(CONN_HOSTNAME_RESOLUTION_ERROR, getMessage(ERR_CONN_EXCEPT, e.what()));
@@ -152,7 +137,7 @@ connectionStatus_t DrillClientImpl::connect(const char* host, const char* port){
     // set socket keep alive
     boost::asio::socket_base::keep_alive keepAlive(true);
     m_socket.set_option(keepAlive);
-	// set no_delay
+    // set no_delay
     boost::asio::ip::tcp::no_delay noDelay(true);
     m_socket.set_option(noDelay);
 
@@ -160,7 +145,7 @@ connectionStatus_t DrillClientImpl::connect(const char* host, const char* port){
     connectedHost << "id: " << m_socket.native_handle() << " address: " << host << ":" << port;
     m_connectedHost = connectedHost.str();
     DRILL_MT_LOG(DRILL_LOG(LOG_INFO) << "Connected to endpoint: " << m_connectedHost << std::endl;)
-    
+
     return CONN_SUCCESS;
 }
 
@@ -180,7 +165,7 @@ connectionStatus_t DrillClientImpl::sendHeartbeat(){
     connectionStatus_t status=CONN_SUCCESS;
     exec::rpc::Ack ack;
     ack.set_ok(true);
-    OutBoundRpcMessage heartbeatMsg(exec::rpc::PING, exec::user::ACK/*can be anything */, 0, &ack);
+    rpc::OutBoundRpcMessage heartbeatMsg(exec::rpc::PING, exec::user::ACK/*can be anything */, 0, &ack);
     boost::lock_guard<boost::mutex> prLock(this->m_prMutex);
     boost::lock_guard<boost::mutex> lock(m_dcMutex);
     DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Heartbeat sent." << std::endl;)
@@ -203,7 +188,7 @@ void DrillClientImpl::handleHeartbeatTimeout(const boost::system::error_code & e
     DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "DrillClientImpl:: Heartbeat timer expired." << std::endl;)
     if(err != boost::asio::error::operation_aborted){
         // Check whether the deadline has passed.
-        DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "DrillClientImpl::Heartbeat Timer -  Expires at: " 
+        DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "DrillClientImpl::Heartbeat Timer -  Expires at: "
             << to_simple_string(m_heartbeatTimer.expires_at())
             << " and time now is: "
             << to_simple_string(boost::asio::deadline_timer::traits_type::now())
@@ -231,8 +216,8 @@ void DrillClientImpl::Close() {
 }
 
 
-connectionStatus_t DrillClientImpl::sendSync(OutBoundRpcMessage& msg){
-    DrillClientImpl::s_encoder.Encode(m_wbuf, msg);
+connectionStatus_t DrillClientImpl::sendSync(rpc::OutBoundRpcMessage& msg){
+    encode(m_wbuf, msg);
     boost::system::error_code ec;
     size_t s=m_socket.write_some(boost::asio::buffer(m_wbuf), ec);
     if(!ec && s!=0){
@@ -292,9 +277,9 @@ void DrillClientImpl::handleHandshake(ByteBuf_t _buf,
     m_deadlineTimer.cancel();
     DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Deadline timer cancelled." << std::endl;)
     if(!error){
-        InBoundRpcMessage msg;
+        rpc::InBoundRpcMessage msg;
         uint32_t length = 0;
-        int bytes_read = DrillClientImpl::s_decoder.LengthDecode(m_rbuf, &length);
+        std::size_t bytes_read = rpc::lengthDecode(m_rbuf, length);
         if(length>0){
             size_t leftover = LEN_PREFIX_BUFLEN - bytes_read;
             ByteBuf_t b=m_rbuf + LEN_PREFIX_BUFLEN;
@@ -309,7 +294,11 @@ void DrillClientImpl::handleHandshake(ByteBuf_t _buf,
                 bytesToRead-=dataBytesRead;
                 b+=dataBytesRead;
             }
-            DrillClientImpl::s_decoder.Decode(m_rbuf+bytes_read, length, msg);
+            if (!decode(m_rbuf+bytes_read, length, msg)) {
+                DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "DrillClientImpl::handleHandshake: ERR_CONN_RDFAIL. Cannot decode handshake.\n";)
+                handleConnError(CONN_FAILURE, getMessage(ERR_CONN_RDFAIL, "Cannot decode handshake"));
+                return;
+            }
         }else{
             DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "DrillClientImpl::handleHandshake: ERR_CONN_RDFAIL. No handshake.\n";)
             handleConnError(CONN_FAILURE, getMessage(ERR_CONN_RDFAIL, "No handshake"));
@@ -321,6 +310,7 @@ void DrillClientImpl::handleHandshake(ByteBuf_t _buf,
         this->m_handshakeStatus=b2u.status();
         this->m_handshakeErrorId=b2u.errorid();
         this->m_handshakeErrorMsg=b2u.errormessage();
+        this->m_serverInfos = b2u.server_infos();
 
     }else{
         // boost error
@@ -362,6 +352,14 @@ connectionStatus_t DrillClientImpl::validateHandshake(DrillUserProperties* prope
     u2b.set_support_listening(true);
     u2b.set_support_timeout(true);
 
+    // Adding version info
+    exec::user::RpcEndpointInfos* infos = u2b.mutable_client_infos();
+    infos->set_name(DRILL_CONNECTOR_NAME);
+    infos->set_version(DRILL_VERSION_STRING);
+    infos->set_majorversion(DRILL_VERSION_MAJOR);
+    infos->set_minorversion(DRILL_VERSION_MINOR);
+    infos->set_patchversion(DRILL_VERSION_PATCH);
+
     if(properties != NULL && properties->size()>0){
         std::string username;
         std::string err;
@@ -374,7 +372,7 @@ connectionStatus_t DrillClientImpl::validateHandshake(DrillUserProperties* prope
         for(size_t i=0; i<properties->size(); i++){
             std::map<std::string,uint32_t>::const_iterator it=DrillUserProperties::USER_PROPERTIES.find(properties->keyAt(i));
             if(it==DrillUserProperties::USER_PROPERTIES.end()){
-                DRILL_MT_LOG(DRILL_LOG(LOG_WARNING) << "Connection property ("<< properties->keyAt(i) 
+                DRILL_MT_LOG(DRILL_LOG(LOG_WARNING) << "Connection property ("<< properties->keyAt(i)
                     << ") is unknown and is being skipped" << std::endl;)
                 continue;
             }
@@ -402,7 +400,7 @@ connectionStatus_t DrillClientImpl::validateHandshake(DrillUserProperties* prope
         boost::lock_guard<boost::mutex> lock(this->m_dcMutex);
         uint64_t coordId = this->getNextCoordinationId();
 
-        OutBoundRpcMessage out_msg(exec::rpc::REQUEST, exec::user::HANDSHAKE, coordId, &u2b);
+        rpc::OutBoundRpcMessage out_msg(exec::rpc::REQUEST, exec::user::HANDSHAKE, coordId, &u2b);
         sendSync(out_msg);
         DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Sent handshake request message. Coordination id: " << coordId << "\n";)
     }
@@ -469,38 +467,159 @@ DrillClientQueryResult* DrillClientImpl::SubmitQuery(::exec::shared::QueryType t
     query.set_type(t);
     query.set_plan(plan);
 
-    uint64_t coordId;
-    DrillClientQueryResult* pQuery=NULL;
+    boost::function<DrillClientQueryResult*(int32_t)> factory = boost::bind(
+            boost::factory<DrillClientQueryResult*>(),
+            boost::ref(*this),
+            _1,
+            boost::cref(plan),
+            l,
+            lCtx);
+    return sendMsg(factory, ::exec::user::RUN_QUERY, query);
+}
+
+DrillClientPrepareHandle* DrillClientImpl::PrepareQuery(const std::string& plan,
+        pfnPreparedStatementListener l,
+        void* lCtx){
+    exec::user::CreatePreparedStatementReq query;
+    query.set_sql_query(plan);
+
+    boost::function<DrillClientPrepareHandle*(int32_t)> factory = boost::bind(
+            boost::factory<DrillClientPrepareHandle*>(),
+            boost::ref(*this),
+            _1,
+            boost::cref(plan),
+            l,
+            lCtx);
+    return sendMsg(factory, ::exec::user::CREATE_PREPARED_STATEMENT, query);
+}
+
+DrillClientQueryResult* DrillClientImpl::ExecuteQuery(const PreparedStatement& pstmt,
+        pfnQueryResultsListener l,
+        void* lCtx){
+    const DrillClientPrepareHandle& handle = static_cast<const DrillClientPrepareHandle&>(pstmt);
+
+    exec::user::RunQuery query;
+    query.set_results_mode(exec::user::STREAM_FULL);
+    query.set_type(::exec::shared::PREPARED_STATEMENT);
+    query.set_allocated_prepared_statement_handle(new ::exec::user::PreparedStatementHandle(handle.m_preparedStatementHandle));
+
+    boost::function<DrillClientQueryResult*(int32_t)> factory = boost::bind(
+            boost::factory<DrillClientQueryResult*>(),
+            boost::ref(*this),
+            _1,
+            boost::cref(handle.m_query),
+            l,
+            lCtx);
+    return sendMsg(factory, ::exec::user::RUN_QUERY, query);
+}
+
+DrillClientCatalogResult* DrillClientImpl::getCatalogs(const std::string& catalogPattern,
+        Metadata::pfnCatalogMetadataListener listener,
+        void* listenerCtx) {
+    exec::user::GetCatalogsReq query;
+    exec::user::LikeFilter* catalogFilter(query.mutable_catalog_name_filter());
+    catalogFilter->set_pattern(catalogPattern);
+
+    boost::function<DrillClientCatalogResult*(int32_t)> factory = boost::bind(
+            boost::factory<DrillClientCatalogResult*>(),
+            boost::ref(*this),
+            _1,
+            listener,
+            listenerCtx);
+    return sendMsg(factory, ::exec::user::GET_CATALOGS, query);
+}
+
+DrillClientSchemaResult* DrillClientImpl::getSchemas(const std::string& catalogPattern,
+        const std::string& schemaPattern,
+        Metadata::pfnSchemaMetadataListener listener,
+        void* listenerCtx) {
+    exec::user::GetSchemasReq query;
+    query.mutable_catalog_name_filter()->set_pattern(catalogPattern);
+    query.mutable_schema_name_filter()->set_pattern(schemaPattern);
+
+    boost::function<DrillClientSchemaResult*(int32_t)> factory = boost::bind(
+            boost::factory<DrillClientSchemaResult*>(),
+            boost::ref(*this),
+            _1,
+            listener,
+            listenerCtx);
+    return sendMsg(factory, ::exec::user::GET_SCHEMAS, query);
+}
+
+DrillClientTableResult* DrillClientImpl::getTables(const std::string& catalogPattern,
+        const std::string& schemaPattern,
+        const std::string& tablePattern,
+		const std::vector<std::string>* tableTypes,
+        Metadata::pfnTableMetadataListener listener,
+        void* listenerCtx) {
+    exec::user::GetTablesReq query;
+    query.mutable_catalog_name_filter()->set_pattern(catalogPattern);
+    query.mutable_schema_name_filter()->set_pattern(schemaPattern);
+    query.mutable_table_name_filter()->set_pattern(tablePattern);
+    if (tableTypes) {
+    	std::copy(tableTypes->begin(), tableTypes->end(),
+    			google::protobuf::RepeatedFieldBackInserter(query.mutable_table_type_filter()));
+    }
+
+    boost::function<DrillClientTableResult*(int32_t)> factory = boost::bind(
+            boost::factory<DrillClientTableResult*>(),
+            boost::ref(*this),
+            _1,
+            listener,
+            listenerCtx);
+    return sendMsg(factory, ::exec::user::GET_TABLES, query);
+}
+
+DrillClientColumnResult* DrillClientImpl::getColumns(const std::string& catalogPattern,
+        const std::string& schemaPattern,
+        const std::string& tablePattern,
+        const std::string& columnsPattern,
+        Metadata::pfnColumnMetadataListener listener,
+        void* listenerCtx) {
+    exec::user::GetColumnsReq query;
+    query.mutable_catalog_name_filter()->set_pattern(catalogPattern);
+    query.mutable_schema_name_filter()->set_pattern(schemaPattern);
+    query.mutable_table_name_filter()->set_pattern(tablePattern);
+    query.mutable_column_name_filter()->set_pattern(columnsPattern);
+
+    boost::function<DrillClientColumnResult*(int32_t)> factory = boost::bind(
+            boost::factory<DrillClientColumnResult*>(),
+            boost::ref(*this),
+            _1,
+            listener,
+            listenerCtx);
+    return sendMsg(factory, ::exec::user::GET_COLUMNS, query);
+}
+
+template<typename Handle>
+Handle* DrillClientImpl::sendMsg(boost::function<Handle*(int32_t)> handleFactory, ::exec::user::RpcType type, const ::google::protobuf::Message& message) {
+    int32_t coordId;
+    Handle* phandle=NULL;
     connectionStatus_t cStatus=CONN_SUCCESS;
     {
         boost::lock_guard<boost::mutex> prLock(this->m_prMutex);
         boost::lock_guard<boost::mutex> dcLock(this->m_dcMutex);
         coordId = this->getNextCoordinationId();
-        OutBoundRpcMessage out_msg(exec::rpc::REQUEST, exec::user::RUN_QUERY, coordId, &query);
+        rpc::OutBoundRpcMessage out_msg(exec::rpc::REQUEST, type, coordId, &message);
 
-        // Create the result object and register the listener before we send the query
-        // because sometimes the caller is not checking the status of the submitQuery call.
-        // This way, the broadcast error call will cause the results listener to be called
-        // with a COMM_ERROR status.
-        pQuery = new DrillClientQueryResult(this, coordId, plan);
-        pQuery->registerListener(l, lCtx);
-        this->m_queryIds[coordId]=pQuery;
+        phandle = handleFactory(coordId);
+        this->m_queryHandles[coordId]=phandle;
 
         connectionStatus_t cStatus=sendSync(out_msg);
         if(cStatus == CONN_SUCCESS){
             bool sendRequest=false;
 
-            DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG)  << "Sent query request. " << "[" << m_connectedHost << "]"  << "Coordination id = " << coordId << std::endl;)
-                DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG)  << "Sent query " <<  "Coordination id = " << coordId << " query: " << plan << std::endl;)
+            DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG)  << "Sent " << ::exec::user::RpcType_Name(type) << " request. " << "[" << m_connectedHost << "]"  << "Coordination id = " << coordId << std::endl;)
+                DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG)  << "Sent " << ::exec::user::RpcType_Name(type) <<  " Coordination id = " << coordId << " query: " << phandle->getQuery() << std::endl;)
 
                 if(m_pendingRequests++==0){
                     sendRequest=true;
                 }else{
-                    DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << "Queueing query request to server" << std::endl;)
+                    DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << "Queuing " << ::exec::user::RpcType_Name(type) <<  " request to server" << std::endl;)
                         DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << "Number of pending requests = " << m_pendingRequests << std::endl;)
                 }
             if(sendRequest){
-                DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << "Sending query request. Number of pending requests = "
+                DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << "Sending " << ::exec::user::RpcType_Name(type) <<  " request. Number of pending requests = "
                         << m_pendingRequests << std::endl;)
                     getNextResult(); // async wait for results
             }
@@ -508,21 +627,18 @@ DrillClientQueryResult* DrillClientImpl::SubmitQuery(::exec::shared::QueryType t
 
     }
     if(cStatus!=CONN_SUCCESS){
-        this->m_queryIds.erase(coordId);
-        delete pQuery;
+        this->m_queryHandles.erase(coordId);
+        delete phandle;
         return NULL;
     }
 
-
-
     //run this in a new thread
     startMessageListener();
 
-    return pQuery;
+    return phandle;
 }
 
 void DrillClientImpl::getNextResult(){
-
     // This call is always made from within a function where the mutex has already been acquired
     //boost::lock_guard<boost::mutex> lock(this->m_dcMutex);
 
@@ -533,7 +649,7 @@ void DrillClientImpl::getNextResult(){
             AllocatedBuffer::s_memCV.wait(memLock);
         }
     }
-    
+
     //use free, not delete to free
     ByteBuf_t readBuf = Utils::allocateBuffer(LEN_PREFIX_BUFLEN);
     if (DrillClientConfig::getQueryTimeout() > 0){
@@ -577,8 +693,7 @@ void DrillClientImpl::waitForResults(){
 
 status_t DrillClientImpl::readMsg(ByteBuf_t _buf,
         AllocatedBufferPtr* allocatedBuffer,
-        InBoundRpcMessage& msg,
-        boost::system::error_code& error){
+        rpc::InBoundRpcMessage& msg){
 
     DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "DrillClientImpl::readMsg: Read message from buffer "
         <<  reinterpret_cast<int*>(_buf) << std::endl;)
@@ -590,7 +705,7 @@ status_t DrillClientImpl::readMsg(ByteBuf_t _buf,
         // We need to protect the readLength and read buffer, and the pending requests counter,
         // but we don't have to keep the lock while we decode the rest of the buffer.
         boost::lock_guard<boost::mutex> lock(this->m_dcMutex);
-        int bytes_read = DrillClientImpl::s_decoder.LengthDecode(_buf, &rmsgLen);
+        std::size_t bytes_read = rpc::lengthDecode(_buf, rmsgLen);
         DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "len bytes = " << bytes_read << std::endl;)
         DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "rmsgLen = " << rmsgLen << std::endl;)
 
@@ -612,7 +727,7 @@ status_t DrillClientImpl::readMsg(ByteBuf_t _buf,
                 << (rmsgLen - leftover) << std::endl;)
             ByteBuf_t b=currentBuffer->m_pBuffer + leftover;
             size_t bytesToRead=rmsgLen - leftover;
-              
+            boost::system::error_code error;
             while(1){
                 size_t dataBytesRead=this->m_socket.read_some(
                         boost::asio::buffer(b, bytesToRead),
@@ -623,10 +738,14 @@ status_t DrillClientImpl::readMsg(ByteBuf_t _buf,
                 bytesToRead-=dataBytesRead;
                 b+=dataBytesRead;
             }
-            
+
             if(!error){
                 // read data successfully
-                DrillClientImpl::s_decoder.Decode(currentBuffer->m_pBuffer, rmsgLen, msg);
+                if (!decode(currentBuffer->m_pBuffer, rmsgLen, msg)) {
+                    Utils::freeBuffer(_buf, LEN_PREFIX_BUFLEN);
+                    return handleQryError(QRY_COMM_ERROR,
+                            getMessage(ERR_QRY_COMMERR, "Cannot decode server message"), NULL);;
+                }
                 DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Done decoding chunk. Coordination id: " <<msg.m_coord_id<< std::endl;)
             }else{
                 Utils::freeBuffer(_buf, LEN_PREFIX_BUFLEN);
@@ -645,7 +764,7 @@ status_t DrillClientImpl::readMsg(ByteBuf_t _buf,
     return QRY_SUCCESS;
 }
 
-status_t DrillClientImpl::processQueryResult(AllocatedBufferPtr  allocatedBuffer, InBoundRpcMessage& msg ){
+status_t DrillClientImpl::processQueryResult(AllocatedBufferPtr  allocatedBuffer, const rpc::InBoundRpcMessage& msg ){
     DrillClientQueryResult* pDrillClientQueryResult=NULL;
     status_t ret=QRY_SUCCESS;
     exec::shared::QueryId qid;
@@ -657,15 +776,15 @@ status_t DrillClientImpl::processQueryResult(AllocatedBufferPtr  allocatedBuffer
         DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << "Processing Query Result " << std::endl;)
         qr.ParseFromArray(msg.m_pbody.data(), msg.m_pbody.size());
         DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << qr.DebugString() << std::endl;)
-        
+
         qid.CopyFrom(qr.query_id());
-        
+
         if (qr.has_query_state() &&
                 qr.query_state() != exec::shared::QueryResult_QueryState_RUNNING &&
                 qr.query_state() != exec::shared::QueryResult_QueryState_STARTING) {
             pDrillClientQueryResult=findQueryResult(qid);
-            //Queries that have been cancelled or whose resources are freed before completion 
-            //do not have a DrillClientQueryResult object. We need not handle the terminal message 
+            //Queries that have been cancelled or whose resources are freed before completion
+            //do not have a DrillClientQueryResult object. We need not handle the terminal message
             //in that case since all it does is to free resources (and they have already been freed)
             if(pDrillClientQueryResult!=NULL){
                 //Validate the RPC message
@@ -703,10 +822,10 @@ status_t DrillClientImpl::processQueryResult(AllocatedBufferPtr  allocatedBuffer
     return ret;
 }
 
-status_t DrillClientImpl::processQueryData(AllocatedBufferPtr  allocatedBuffer, InBoundRpcMessage& msg ){
+status_t DrillClientImpl::processQueryData(AllocatedBufferPtr allocatedBuffer, const rpc::InBoundRpcMessage& msg ){
     DrillClientQueryResult* pDrillClientQueryResult=NULL;
     status_t ret=QRY_SUCCESS;
-    exec::shared::QueryId qid;
+    ::exec::shared::QueryId qid;
     // Be a good client and send ack as early as possible.
     // Drillbit pushed the query result to the client, the client should send ack
     // whenever it receives the message
@@ -720,7 +839,7 @@ status_t DrillClientImpl::processQueryData(AllocatedBufferPtr  allocatedBuffer,
         qr->ParseFromArray(msg.m_pbody.data(), msg.m_pbody.size());
         DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << qr->DebugString() << std::endl;)
 
-        qid.CopyFrom(qr->query_id());
+        qid = ::exec::shared::QueryId(qr->query_id());
         if(qid.part1()==0){
             DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "DrillClientImpl::processQueryData: QID=0. Ignore and return QRY_SUCCESS." << std::endl;)
             delete allocatedBuffer;
@@ -729,13 +848,13 @@ status_t DrillClientImpl::processQueryData(AllocatedBufferPtr  allocatedBuffer,
 
         pDrillClientQueryResult=findQueryResult(qid);
         if(pDrillClientQueryResult==NULL){
-            DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Cleaning up resources allocated for canceled query (" 
+            DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Cleaning up resources allocated for canceled query ("
                                  << debugPrintQid(qid) << ")." << std::endl;)
             delete qr;
             delete allocatedBuffer;
             return ret;
         }
-        
+
         //Validate the RPC message
         std::string valErr;
         if( (ret=validateDataMessage(msg, *qr, valErr)) != QRY_SUCCESS){
@@ -765,20 +884,13 @@ status_t DrillClientImpl::processQueryData(AllocatedBufferPtr  allocatedBuffer,
         }
 
         pDrillClientQueryResult->setIsQueryPending(true);
-        pfnQueryResultsListener pResultsListener=pDrillClientQueryResult->m_pResultsListener;
         if(pDrillClientQueryResult->m_bIsLastChunk){
             DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << debugPrintQid(*pDrillClientQueryResult->m_pQueryId)
                 <<  "Received last batch. " << std::endl;)
             ret=QRY_NO_MORE_DATA;
         }
         pDrillClientQueryResult->setQueryStatus(ret);
-        if(pResultsListener!=NULL){
-            ret = pResultsListener(pDrillClientQueryResult, pRecordBatch, NULL);
-        }else{
-            //Use a default callback that is called when a record batch is received
-            ret = pDrillClientQueryResult->defaultQueryResultsListener(pDrillClientQueryResult,
-                    pRecordBatch, NULL);
-        }
+        ret = pDrillClientQueryResult->notifyListener(pRecordBatch, NULL);
     } // release lock
     if(ret==QRY_FAILURE){
         sendCancel(&qid);
@@ -787,31 +899,37 @@ status_t DrillClientImpl::processQueryData(AllocatedBufferPtr  allocatedBuffer,
         pDrillClientQueryResult->setIsQueryPending(false);
         DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << "Client app cancelled query." << std::endl;)
         pDrillClientQueryResult->setQueryStatus(ret);
-        clearMapEntries(pDrillClientQueryResult);
+        removeQueryHandle(pDrillClientQueryResult);
+        removeQueryResult(pDrillClientQueryResult);
         return ret;
     }
     return ret;
 }
 
-status_t DrillClientImpl::processQueryId(AllocatedBufferPtr allocatedBuffer, InBoundRpcMessage& msg ){
-    DrillClientQueryResult* pDrillClientQueryResult=NULL;
+status_t DrillClientImpl::processQueryId(AllocatedBufferPtr allocatedBuffer, const rpc::InBoundRpcMessage& msg ){
     DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << "Processing Query Handle with coordination id:" << msg.m_coord_id << std::endl;)
     status_t ret=QRY_SUCCESS;
 
+    // make sure to deallocate buffer
+    boost::shared_ptr<AllocatedBuffer> deallocationGuard(allocatedBuffer);
     boost::lock_guard<boost::mutex> lock(m_dcMutex);
-    std::map<int,DrillClientQueryResult*>::iterator it;
-    for(it=this->m_queryIds.begin();it!=this->m_queryIds.end();it++){
-        std::string qidString = it->second->m_pQueryId!=NULL?debugPrintQid(*it->second->m_pQueryId):std::string("NULL");
-        DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << "DrillClientImpl::processQueryId: m_queryIds: coordinationId: " << it->first
+    for(std::map< ::exec::shared::QueryId*, DrillClientQueryResult*>::const_iterator it=this->m_queryResults.begin();it!=this->m_queryResults.end();it++){
+        DrillClientQueryResult* pDrillClientQueryResult=it->second;
+        std::string qidString = (pDrillClientQueryResult->m_pQueryId!=NULL)?debugPrintQid(*pDrillClientQueryResult->m_pQueryId):std::string("NULL");
+        DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << "DrillClientImpl::processQueryId: m_queryIds: coordinationId: " << pDrillClientQueryResult->m_coordinationId
         << " QueryId: "<< qidString << std::endl;)
     }
     if(msg.m_coord_id==0){
         DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "DrillClientImpl::processQueryId: m_coord_id=0. Ignore and return QRY_SUCCESS." << std::endl;)
         return QRY_SUCCESS;
     }
-    it=this->m_queryIds.find(msg.m_coord_id);
-    if(it!=this->m_queryIds.end()){
-        pDrillClientQueryResult=(*it).second;
+    std::map<int, DrillClientQueryHandle*>::const_iterator it;
+    it=this->m_queryHandles.find(msg.m_coord_id);
+    if(it!=this->m_queryHandles.end()){
+        DrillClientQueryResult* pDrillClientQueryResult=dynamic_cast<DrillClientQueryResult*>((*it).second);
+        if (!pDrillClientQueryResult) {
+            return handleQryError(QRY_INTERNAL_ERROR, getMessage(ERR_QRY_INVQUERYID), NULL);
+        }
         exec::shared::QueryId *qid = new exec::shared::QueryId;
         DRILL_MT_LOG(DRILL_LOG(LOG_TRACE)  << "Received Query Handle " << msg.m_pbody.size() << std::endl;)
         qid->ParseFromArray(msg.m_pbody.data(), msg.m_pbody.size());
@@ -820,14 +938,241 @@ status_t DrillClientImpl::processQueryId(AllocatedBufferPtr allocatedBuffer, InB
         //save queryId allocated here so we can free it later
         pDrillClientQueryResult->setQueryId(qid);
     }else{
-        delete allocatedBuffer;
         return handleQryError(QRY_INTERNAL_ERROR, getMessage(ERR_QRY_INVQUERYID), NULL);
     }
-    delete allocatedBuffer;
     return ret;
 }
 
-DrillClientQueryResult* DrillClientImpl::findQueryResult(exec::shared::QueryId& qid){
+status_t DrillClientImpl::processPreparedStatement(AllocatedBufferPtr allocatedBuffer, const rpc::InBoundRpcMessage& msg ){
+    DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << "Processing Prepared Statement with coordination id:" << msg.m_coord_id << std::endl;)
+    status_t ret=QRY_SUCCESS;
+
+    // make sure to deallocate buffer
+    boost::shared_ptr<AllocatedBuffer> deallocationGuard(allocatedBuffer);
+    boost::lock_guard<boost::mutex> lock(m_dcMutex);
+
+    if(msg.m_coord_id==0){
+        DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "DrillClientImpl::processPreparedStatement: m_coord_id=0. Ignore and return QRY_SUCCESS." << std::endl;)
+        return QRY_SUCCESS;
+    }
+    std::map<int,DrillClientQueryHandle*>::const_iterator it=this->m_queryHandles.find(msg.m_coord_id);
+    if(it!=this->m_queryHandles.end()){
+        DrillClientPrepareHandle* pDrillClientPrepareHandle=static_cast<DrillClientPrepareHandle*>((*it).second);
+        exec::user::CreatePreparedStatementResp resp;
+        DRILL_MT_LOG(DRILL_LOG(LOG_TRACE)  << "Received Prepared Statement Handle " << msg.m_pbody.size() << std::endl;)
+        if (!resp.ParseFromArray(msg.m_pbody.data(), msg.m_pbody.size())) {
+            return handleQryError(QRY_COMM_ERROR, "Cannot decode prepared statement", pDrillClientPrepareHandle);
+        }
+        if (resp.has_status() && resp.status() != exec::user::OK) {
+            return handleQryError(QRY_FAILED, resp.error(), pDrillClientPrepareHandle);
+        }
+        pDrillClientPrepareHandle->setupPreparedStatement(resp.prepared_statement());
+        pDrillClientPrepareHandle->notifyListener(pDrillClientPrepareHandle, NULL);
+        DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << "Prepared Statement handle - " << resp.prepared_statement().server_handle().DebugString() << std::endl;)
+    }else{
+        return handleQryError(QRY_INTERNAL_ERROR, getMessage(ERR_QRY_INVQUERYID), NULL);
+    }
+    m_pendingRequests--;
+    DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "DrillClientImpl::processPreparedStament: " << m_pendingRequests << " requests pending." << std::endl;)
+    if(m_pendingRequests==0){
+        // signal any waiting client that it can exit because there are no more any query results to arrive.
+        // We keep the heartbeat going though.
+        m_cv.notify_one();
+    }
+    return ret;
+}
+
+status_t DrillClientImpl::processCatalogsResult(AllocatedBufferPtr allocatedBuffer, const rpc::InBoundRpcMessage& msg ){
+    DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << "Processing GetCatalogsResp with coordination id:" << msg.m_coord_id << std::endl;)
+    status_t ret=QRY_SUCCESS;
+
+    // make sure to deallocate buffer
+    boost::shared_ptr<AllocatedBuffer> deallocationGuard(allocatedBuffer);
+    boost::lock_guard<boost::mutex> lock(m_dcMutex);
+
+    if(msg.m_coord_id==0){
+        DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "DrillClientImpl::processCatalogsResult: m_coord_id=0. Ignore and return QRY_SUCCESS." << std::endl;)
+        return QRY_SUCCESS;
+    }
+    std::map<int,DrillClientQueryHandle*>::const_iterator it=this->m_queryHandles.find(msg.m_coord_id);
+    if(it!=this->m_queryHandles.end()){
+        DrillClientCatalogResult* pHandle=static_cast<DrillClientCatalogResult*>((*it).second);
+        exec::user::GetCatalogsResp* resp = new exec::user::GetCatalogsResp;
+        pHandle->attachMetadataResult(resp);
+        DRILL_MT_LOG(DRILL_LOG(LOG_TRACE)  << "Received GetCatalogs result Handle " << msg.m_pbody.size() << std::endl;)
+        if (!(resp->ParseFromArray(msg.m_pbody.data(), msg.m_pbody.size()))) {
+            return handleQryError(QRY_COMM_ERROR, "Cannot decode getcatalogs results", pHandle);
+        }
+        if (resp->status() != exec::user::OK) {
+            return handleQryError(QRY_FAILED, resp->error(), pHandle);
+        }
+
+        const ::google::protobuf::RepeatedPtrField< ::exec::user::CatalogMetadata>& catalogs = resp->catalogs();
+        pHandle->m_meta.clear();
+        pHandle->m_meta.reserve(resp->catalogs_size());
+
+        for(::google::protobuf::RepeatedPtrField< ::exec::user::CatalogMetadata>::const_iterator it = catalogs.begin(); it != catalogs.end(); ++it) {
+            meta::DrillCatalogMetadata meta(*it);
+            pHandle->m_meta.push_back(meta);
+        }
+        pHandle->notifyListener(&pHandle->m_meta, NULL);
+        DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << "GetCatalogs result -  " << resp->catalogs_size() << " catalog(s)" << std::endl;)
+    }else{
+        return handleQryError(QRY_INTERNAL_ERROR, getMessage(ERR_QRY_INVQUERYID), NULL);
+    }
+    m_pendingRequests--;
+    DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "DrillClientImpl::processCatalogsResult: " << m_pendingRequests << " requests pending." << std::endl;)
+    if(m_pendingRequests==0){
+        // signal any waiting client that it can exit because there are no more any query results to arrive.
+        // We keep the heartbeat going though.
+        m_cv.notify_one();
+    }
+    return ret;
+}
+
+status_t DrillClientImpl::processSchemasResult(AllocatedBufferPtr allocatedBuffer, const rpc::InBoundRpcMessage& msg ){
+    DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << "Processing GetSchemaResp with coordination id:" << msg.m_coord_id << std::endl;)
+    status_t ret=QRY_SUCCESS;
+
+    // make sure to deallocate buffer
+    boost::shared_ptr<AllocatedBuffer> deallocationGuard(allocatedBuffer);
+    boost::lock_guard<boost::mutex> lock(m_dcMutex);
+
+    if(msg.m_coord_id==0){
+        DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "DrillClientImpl::processSchemasResult: m_coord_id=0. Ignore and return QRY_SUCCESS." << std::endl;)
+        return QRY_SUCCESS;
+    }
+    std::map<int,DrillClientQueryHandle*>::const_iterator it=this->m_queryHandles.find(msg.m_coord_id);
+    if(it!=this->m_queryHandles.end()){
+        DrillClientSchemaResult* pHandle=static_cast<DrillClientSchemaResult*>((*it).second);
+        exec::user::GetSchemasResp* resp = new exec::user::GetSchemasResp();
+        pHandle->attachMetadataResult(resp);
+        DRILL_MT_LOG(DRILL_LOG(LOG_TRACE)  << "Received GetSchemasResp result Handle " << msg.m_pbody.size() << std::endl;)
+        if (!(resp->ParseFromArray(msg.m_pbody.data(), msg.m_pbody.size()))) {
+            return handleQryError(QRY_COMM_ERROR, "Cannot decode getschemas results", pHandle);
+        }
+        if (resp->status() != exec::user::OK) {
+            return handleQryError(QRY_FAILED, resp->error(), pHandle);
+        }
+
+        const ::google::protobuf::RepeatedPtrField< ::exec::user::SchemaMetadata>& schemas = resp->schemas();
+        pHandle->m_meta.clear();
+        pHandle->m_meta.reserve(resp->schemas_size());
+
+        for(::google::protobuf::RepeatedPtrField< ::exec::user::SchemaMetadata>::const_iterator it = schemas.begin(); it != schemas.end(); ++it) {
+            meta::DrillSchemaMetadata meta(*it);
+            pHandle->m_meta.push_back(meta);
+        }
+        pHandle->notifyListener(&pHandle->m_meta, NULL);
+        DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << "GetSchemaResp result - " << resp->schemas_size() << " schema(s)" << std::endl;)
+    }else{
+        return handleQryError(QRY_INTERNAL_ERROR, getMessage(ERR_QRY_INVQUERYID), NULL);
+    }
+    m_pendingRequests--;
+    DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "DrillClientImpl::processSchemasResult: " << m_pendingRequests << " requests pending." << std::endl;)
+    if(m_pendingRequests==0){
+        // signal any waiting client that it can exit because there are no more any query results to arrive.
+        // We keep the heartbeat going though.
+        m_cv.notify_one();
+    }
+    return ret;
+}
+
+status_t DrillClientImpl::processTablesResult(AllocatedBufferPtr allocatedBuffer, const rpc::InBoundRpcMessage& msg ){
+    DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << "Processing GetTablesResp with coordination id:" << msg.m_coord_id << std::endl;)
+    status_t ret=QRY_SUCCESS;
+
+    // make sure to deallocate buffer
+    boost::shared_ptr<AllocatedBuffer> deallocationGuard(allocatedBuffer);
+    boost::lock_guard<boost::mutex> lock(m_dcMutex);
+
+    if(msg.m_coord_id==0){
+        DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "DrillClientImpl::processTablesResult: m_coord_id=0. Ignore and return QRY_SUCCESS." << std::endl;)
+        return QRY_SUCCESS;
+    }
+    std::map<int,DrillClientQueryHandle*>::const_iterator it=this->m_queryHandles.find(msg.m_coord_id);
+    if(it!=this->m_queryHandles.end()){
+        DrillClientTableResult* pHandle=static_cast<DrillClientTableResult*>((*it).second);
+        exec::user::GetTablesResp* resp =  new exec::user::GetTablesResp();
+        pHandle->attachMetadataResult(resp);
+        DRILL_MT_LOG(DRILL_LOG(LOG_TRACE)  << "Received GeTablesResp result Handle " << msg.m_pbody.size() << std::endl;)
+        if (!(resp->ParseFromArray(msg.m_pbody.data(), msg.m_pbody.size()))) {
+            return handleQryError(QRY_COMM_ERROR, "Cannot decode gettables results", pHandle);
+        }
+        if (resp->status() != exec::user::OK) {
+            return handleQryError(QRY_FAILED, resp->error(), pHandle);
+        }
+        const ::google::protobuf::RepeatedPtrField< ::exec::user::TableMetadata>& tables = resp->tables();
+        pHandle->m_meta.clear();
+        pHandle->m_meta.reserve(resp->tables_size());
+
+        for(::google::protobuf::RepeatedPtrField< ::exec::user::TableMetadata>::const_iterator it = tables.begin(); it != tables.end(); ++it) {
+            meta::DrillTableMetadata meta(*it);
+            pHandle->m_meta.push_back(meta);
+        }
+        pHandle->notifyListener(&pHandle->m_meta, NULL);
+        DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << "GetTables result - " << resp->tables_size() << " table(s)" << std::endl;)
+    }else{
+        return handleQryError(QRY_INTERNAL_ERROR, getMessage(ERR_QRY_INVQUERYID), NULL);
+    }
+    m_pendingRequests--;
+    DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "DrillClientImpl::processTablesResult: " << m_pendingRequests << " requests pending." << std::endl;)
+    if(m_pendingRequests==0){
+        // signal any waiting client that it can exit because there are no more any query results to arrive.
+        // We keep the heartbeat going though.
+        m_cv.notify_one();
+    }
+    return ret;
+}
+
+status_t DrillClientImpl::processColumnsResult(AllocatedBufferPtr allocatedBuffer, const rpc::InBoundRpcMessage& msg ){
+    DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << "Processing GetColumnsResp with coordination id:" << msg.m_coord_id << std::endl;)
+    status_t ret=QRY_SUCCESS;
+
+    // make sure to deallocate buffer
+    boost::shared_ptr<AllocatedBuffer> deallocationGuard(allocatedBuffer);
+    boost::lock_guard<boost::mutex> lock(m_dcMutex);
+
+    if(msg.m_coord_id==0){
+         DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "DrillClientImpl::processColumnsResult: m_coord_id=0. Ignore and return QRY_SUCCESS." << std::endl;)
+        return QRY_SUCCESS;
+    }
+    std::map<int,DrillClientQueryHandle*>::const_iterator it=this->m_queryHandles.find(msg.m_coord_id);
+    if(it!=this->m_queryHandles.end()){
+        DrillClientColumnResult* pHandle=static_cast<DrillClientColumnResult*>((*it).second);
+        exec::user::GetColumnsResp* resp = new exec::user::GetColumnsResp();
+        pHandle->attachMetadataResult(resp);
+        DRILL_MT_LOG(DRILL_LOG(LOG_TRACE)  << "Received GetColumnsResp result Handle " << msg.m_pbody.size() << std::endl;)
+        if (!(resp->ParseFromArray(msg.m_pbody.data(), msg.m_pbody.size()))) {
+            return handleQryError(QRY_COMM_ERROR, "Cannot decode getcolumns results", pHandle);
+        }
+        if (resp->status() != exec::user::OK) {
+            return handleQryError(QRY_FAILED, resp->error(), pHandle);
+        }
+        const ::google::protobuf::RepeatedPtrField< ::exec::user::ColumnMetadata>& columns = resp->columns();
+        pHandle->m_meta.clear();
+        pHandle->m_meta.reserve(resp->columns_size());
+
+        for(::google::protobuf::RepeatedPtrField< ::exec::user::ColumnMetadata>::const_iterator it = columns.begin(); it != columns.end(); ++it) {
+            meta::DrillColumnMetadata meta(*it);
+            pHandle->m_meta.push_back(meta);
+        }
+        pHandle->notifyListener(&pHandle->m_meta, NULL);
+        DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << "GetColumnsResp result - " << resp->columns_size() << " columns(s)" << std::endl;)
+    }else{
+        return handleQryError(QRY_INTERNAL_ERROR, getMessage(ERR_QRY_INVQUERYID), NULL);
+    }
+    m_pendingRequests--;
+    DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "DrillClientImpl::processColumnsResult: " << m_pendingRequests << " requests pending." << std::endl;)
+    if(m_pendingRequests==0){
+        // signal any waiting client that it can exit because there are no more any query results to arrive.
+        // We keep the heartbeat going though.
+        m_cv.notify_one();
+    }
+    return ret;
+}
+
+DrillClientQueryResult* DrillClientImpl::findQueryResult(const exec::shared::QueryId& qid){
     DrillClientQueryResult* pDrillClientQueryResult=NULL;
     DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << "Searching for Query Id - " << debugPrintQid(qid) << std::endl;)
     std::map<exec::shared::QueryId*, DrillClientQueryResult*, compareQueryId>::iterator it;
@@ -838,7 +1183,7 @@ DrillClientQueryResult* DrillClientImpl::findQueryResult(exec::shared::QueryId&
                 << it->first->part2() << "]\n";)
         }
     }
-    it=this->m_queryResults.find(&qid);
+    it=this->m_queryResults.find(const_cast<exec::shared::QueryId * const>(&qid));
     if(it!=this->m_queryResults.end()){
         pDrillClientQueryResult=(*it).second;
         DRILL_MT_LOG(DRILL_LOG(LOG_DEBUG) << "Drill Client Query Result Query Id - " <<
@@ -925,9 +1270,8 @@ void DrillClientImpl::handleReadTimeout(const boost::system::error_code & err){
 }
 
 void DrillClientImpl::handleRead(ByteBuf_t _buf,
-        const boost::system::error_code& err,
+        const boost::system::error_code& error,
         size_t bytes_transferred) {
-    boost::system::error_code error=err;
     DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "DrillClientImpl::handleRead: Handle Read from buffer "
         <<  reinterpret_cast<int*>(_buf) << std::endl;)
     if(DrillClientConfig::getQueryTimeout() > 0){
@@ -935,120 +1279,153 @@ void DrillClientImpl::handleRead(ByteBuf_t _buf,
         DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "DrillClientImpl::handleRead: Cancel deadline timer.\n";)
         m_deadlineTimer.cancel();
     }
-    if(!error){
-        InBoundRpcMessage msg;
-        boost::lock_guard<boost::mutex> lock(this->m_prMutex);
+    if (error) {
+        // boost error
+        Utils::freeBuffer(_buf, LEN_PREFIX_BUFLEN);
+        boost::lock_guard<boost::mutex> lock(this->m_dcMutex);
+        DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "DrillClientImpl::handleRead: ERR_QRY_COMMERR. "
+            "Boost Communication Error: " << error.message() << std::endl;)
+        handleQryError(QRY_COMM_ERROR, getMessage(ERR_QRY_COMMERR, error.message().c_str()), NULL);
+        return;
+    }
 
-        DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Getting new message" << std::endl;)
-        AllocatedBufferPtr allocatedBuffer=NULL;
+    rpc::InBoundRpcMessage msg;
+    boost::lock_guard<boost::mutex> lockPR(this->m_prMutex);
 
-        if(readMsg(_buf, &allocatedBuffer, msg, error)!=QRY_SUCCESS){
-            if(m_pendingRequests!=0){
-                boost::lock_guard<boost::mutex> lock(this->m_dcMutex);
-                getNextResult();
-            }
-            return;
+    DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Getting new message" << std::endl;)
+    AllocatedBufferPtr allocatedBuffer=NULL;
+
+    if(readMsg(_buf, &allocatedBuffer, msg)!=QRY_SUCCESS){
+        delete allocatedBuffer;
+        if(m_pendingRequests!=0){
+            boost::lock_guard<boost::mutex> lock(this->m_dcMutex);
+            getNextResult();
         }
+        return;
+    }
 
-        if(!error && msg.m_mode==exec::rpc::PONG){ //heartbeat response. Throw it away
-            m_pendingRequests--;
+    if(msg.m_mode==exec::rpc::PONG) { //heartbeat response. Throw it away
+        m_pendingRequests--;
+        delete allocatedBuffer;
+        DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Received heartbeat from server. " <<  std::endl;)
+        if(m_pendingRequests!=0){
+            boost::lock_guard<boost::mutex> lock(this->m_dcMutex);
+            getNextResult();
+        }else{
+            boost::unique_lock<boost::mutex> cvLock(this->m_dcMutex);
+            DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "No more results expected from server. " <<  std::endl;)
+            m_cv.notify_one();
+        }
+
+        return;
+    }
+
+    if(msg.m_mode == exec::rpc::RESPONSE) {
+        status_t s;
+        switch(msg.m_rpc_type) {
+        case exec::user::QUERY_HANDLE:
+            s = processQueryId(allocatedBuffer, msg);
+            break;
+
+        case exec::user::PREPARED_STATEMENT:
+            s = processPreparedStatement(allocatedBuffer, msg);
+            break;
+
+        case exec::user::CATALOGS:
+            s = processCatalogsResult(allocatedBuffer, msg);
+            break;
+
+        case exec::user::SCHEMAS:
+            s = processSchemasResult(allocatedBuffer, msg);
+            break;
+
+        case exec::user::TABLES:
+            s = processTablesResult(allocatedBuffer, msg);
+            break;
+
+        case exec::user::COLUMNS:
+            s = processColumnsResult(allocatedBuffer, msg);
+            break;
+
+        case exec::user::HANDSHAKE:
+            DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "DrillClientImpl::handleRead: Handshake response from server. Ignore.\n";)
             delete allocatedBuffer;
-            DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Received heartbeat from server. " <<  std::endl;)
-            if(m_pendingRequests!=0){
-                boost::lock_guard<boost::mutex> lock(this->m_dcMutex);
-                getNextResult();
-            }else{
-                boost::unique_lock<boost::mutex> cvLock(this->m_dcMutex);
-                DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "No more results expected from server. " <<  std::endl;)
-                m_cv.notify_one();
-            }
-            return;
-        }else if(!error && msg.m_rpc_type==exec::user::QUERY_RESULT){
-            status_t s = processQueryResult(allocatedBuffer, msg);
-            if(s !=QRY_SUCCESS && s!= QRY_NO_MORE_DATA){
-                if(m_pendingRequests!=0){
-                    boost::lock_guard<boost::mutex> lock(this->m_dcMutex);
-                    getNextResult();
-                }
-                return;
-            }
-        }else if(!error && msg.m_rpc_type==exec::user::QUERY_DATA){
-            if(processQueryData(allocatedBuffer, msg)!=QRY_SUCCESS){
-                if(m_pendingRequests!=0){
-                    boost::lock_guard<boost::mutex> lock(this->m_dcMutex);
-                    getNextResult();
-                }
-                return;
-            }
-        }else if(!error && msg.m_rpc_type==exec::user::QUERY_HANDLE){
-            if(processQueryId(allocatedBuffer, msg)!=QRY_SUCCESS){
-                if(m_pendingRequests!=0){
-                    boost::lock_guard<boost::mutex> lock(this->m_dcMutex);
-                    getNextResult();
-                }
-                return;
-            }
-        }else if(!error && msg.m_rpc_type==exec::user::ACK){
+            break;
+
+        case exec::user::ACK:
             // Cancel requests will result in an ACK sent back.
             // Consume silently
+            s = QRY_CANCELED;
             delete allocatedBuffer;
-            if(m_pendingRequests!=0){
-                boost::lock_guard<boost::mutex> lock(this->m_dcMutex);
-                getNextResult();
-            }
-            return;
-        }else{
+            break;
+
+        default:
+            DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "DrillClientImpl::handleRead: ERR_QRY_INVRPCTYPE. "
+                    << "QueryResult returned " << msg.m_rpc_type << std::endl;)
+            delete allocatedBuffer;
+            handleQryError(QRY_INTERNAL_ERROR, getMessage(ERR_QRY_INVRPCTYPE, msg.m_rpc_type), NULL);
+        }
+
+        if (m_pendingRequests != 0) {
             boost::lock_guard<boost::mutex> lock(this->m_dcMutex);
-            if(error){
-                // We have a socket read error, but we do not know which query this is for.
-                // Signal ALL pending queries that they should stop waiting.
-                delete allocatedBuffer;
-                DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "read error: " << error << std::endl;)
-                handleQryError(QRY_COMM_ERROR, getMessage(ERR_QRY_COMMERR, error.message().c_str()), NULL);
-                return;
-            }else{
-                // If not QUERY_RESULT, then we think something serious has gone wrong?
-                // In one case when the client hung, we observed that the server was sending a handshake request to the client
-                // We should properly handle these handshake requests/responses
-                if(msg.has_rpc_type() && msg.m_rpc_type==exec::user::HANDSHAKE){
-                    if(msg.has_mode() && msg.m_mode==exec::rpc::REQUEST){
-                        DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "DrillClientImpl::handleRead: Handshake request from server. Send response.\n";)
-                        exec::user::UserToBitHandshake u2b;
-                        u2b.set_channel(exec::shared::USER);
-                        u2b.set_rpc_version(DRILL_RPC_VERSION);
-                        u2b.set_support_listening(true);
-                        OutBoundRpcMessage out_msg(exec::rpc::RESPONSE, exec::user::HANDSHAKE, msg.m_coord_id, &u2b);
-                        sendSync(out_msg);
-                        DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "DrillClientImpl::handleRead: Handshake response sent.\n";)
-                    }else{
-                        DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "DrillClientImpl::handleRead: Handshake response from server. Ignore.\n";)
-                    }
-                }else{
-                    DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "DrillClientImpl::handleRead: ERR_QRY_INVRPCTYPE. "
-                        << "QueryResult returned " << msg.m_rpc_type << std::endl;)
-                    handleQryError(QRY_INTERNAL_ERROR, getMessage(ERR_QRY_INVRPCTYPE, msg.m_rpc_type), NULL);
-                }
-                delete allocatedBuffer;
-                return;
+            getNextResult();
+        }
+
+        return;
+    }
+
+    if (msg.has_mode() && msg.m_mode == exec::rpc::REQUEST) {
+        status_t s;
+        switch(msg.m_rpc_type) {
+        case exec::user::QUERY_RESULT:
+            s = processQueryResult(allocatedBuffer, msg);
+            break;
+
+        case exec::user::QUERY_DATA:
+            s = processQueryData(allocatedBuffer, msg);
+            break;
+
+        case exec::user::HANDSHAKE:
+            DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "DrillClientImpl::handleRead: Handshake request from server. Send response.\n";)
+            delete allocatedBuffer;
+            // In one case when the client hung, we observed that the server was sending a handshake request to the client
+            // We should properly handle these handshake requests/responses
+            {
+                boost::lock_guard<boost::mutex> lockDC(this->m_dcMutex);
+                exec::user::UserToBitHandshake u2b;
+                u2b.set_channel(exec::shared::USER);
+                u2b.set_rpc_version(DRILL_RPC_VERSION);
+                u2b.set_support_listening(true);
+                rpc::OutBoundRpcMessage out_msg(exec::rpc::RESPONSE, exec::user::HANDSHAKE, msg.m_coord_id, &u2b);
+                sendSync(out_msg);
+                DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "DrillClientImpl::handleRead: Handshake response sent.\n";)
             }
+            break;
+
+        default:
+            DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "DrillClientImpl::handleRead: ERR_QRY_INVRPCTYPE. "
+                    << "QueryResult returned " << msg.m_rpc_type << std::endl;)
+            delete allocatedBuffer;
+            handleQryError(QRY_INTERNAL_ERROR, getMessage(ERR_QRY_INVRPCTYPE, msg.m_rpc_type), NULL);
         }
-        {
+
+        if (m_pendingRequests != 0) {
             boost::lock_guard<boost::mutex> lock(this->m_dcMutex);
             getNextResult();
         }
-    }else{
-        // boost error
-        Utils::freeBuffer(_buf, LEN_PREFIX_BUFLEN);
-        boost::lock_guard<boost::mutex> lock(this->m_dcMutex);
-        DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "DrillClientImpl::handleRead: ERR_QRY_COMMERR. "
-            "Boost Communication Error: " << error.message() << std::endl;)
-        handleQryError(QRY_COMM_ERROR, getMessage(ERR_QRY_COMMERR, error.message().c_str()), NULL);
+
         return;
     }
-    return;
+
+    // If not QUERY_RESULT, then we think something serious has gone wrong?
+    DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "DrillClientImpl::handleRead: ERR_QRY_INVRPCTYPE. "
+        << "QueryResult returned " << msg.m_rpc_type << " for " << msg.m_mode << std::endl;)
+    handleQryError(QRY_INTERNAL_ERROR, getMessage(ERR_QRY_INVRPCTYPE, msg.m_rpc_type), NULL);
+    delete allocatedBuffer;
+
 }
 
-status_t DrillClientImpl::validateDataMessage(InBoundRpcMessage& msg, exec::shared::QueryData& qd, std::string& valErr){
+status_t DrillClientImpl::validateDataMessage(const rpc::InBoundRpcMessage& msg, const exec::shared::QueryData& qd, std::string& valErr){
     if(msg.m_mode == exec::rpc::RESPONSE_FAILURE){
         valErr=getMessage(ERR_QRY_RESPFAIL);
         return QRY_FAILURE;
@@ -1060,7 +1437,7 @@ status_t DrillClientImpl::validateDataMessage(InBoundRpcMessage& msg, exec::shar
     return QRY_SUCCESS;
 }
 
-status_t DrillClientImpl::validateResultMessage(InBoundRpcMessage& msg, exec::shared::QueryResult& qr, std::string& valErr){
+status_t DrillClientImpl::validateResultMessage(const rpc::InBoundRpcMessage& msg, const exec::shared::QueryResult& qr, std::string& valErr){
     if(msg.m_mode == exec::rpc::RESPONSE_FAILURE){
         valErr=getMessage(ERR_QRY_RESPFAIL);
         return QRY_FAILURE;
@@ -1072,10 +1449,10 @@ status_t DrillClientImpl::validateResultMessage(InBoundRpcMessage& msg, exec::sh
     return QRY_SUCCESS;
 }
 
-connectionStatus_t DrillClientImpl::handleConnError(connectionStatus_t status, std::string msg){
+connectionStatus_t DrillClientImpl::handleConnError(connectionStatus_t status, const std::string& msg){
     DrillClientError* pErr = new DrillClientError(status, DrillClientError::CONN_ERROR_START+status, msg);
     m_pendingRequests=0;
-    if(!m_queryIds.empty()){
+    if(!m_queryHandles.empty()){
         // set query error only if queries are running
         broadcastError(pErr);
     }else{
@@ -1086,12 +1463,12 @@ connectionStatus_t DrillClientImpl::handleConnError(connectionStatus_t status, s
     return status;
 }
 
-status_t DrillClientImpl::handleQryError(status_t status, std::string msg, DrillClientQueryResult* pQueryResult){
+status_t DrillClientImpl::handleQryError(status_t status, const std::string& msg, DrillClientQueryHandle* pQueryHandle){
     DrillClientError* pErr = new DrillClientError(status, DrillClientError::QRY_ERROR_START+status, msg);
     // set query error only if queries are running
-    if(pQueryResult!=NULL){
+    if(pQueryHandle!=NULL){
         m_pendingRequests--;
-        pQueryResult->signalError(pErr);
+        pQueryHandle->signalError(pErr);
     }else{
         m_pendingRequests=0;
         broadcastError(pErr);
@@ -1101,27 +1478,27 @@ status_t DrillClientImpl::handleQryError(status_t status, std::string msg, Drill
 
 status_t DrillClientImpl::handleQryError(status_t status,
         const exec::shared::DrillPBError& e,
-        DrillClientQueryResult* pQueryResult){
-    assert(pQueryResult!=NULL);
+        DrillClientQueryHandle* pQueryHandle){
+    assert(pQueryHandle!=NULL);
     DrillClientError* pErr =  DrillClientError::getErrorObject(e);
-    pQueryResult->signalError(pErr);
+    pQueryHandle->signalError(pErr);
     m_pendingRequests--;
     return status;
 }
 
 void DrillClientImpl::broadcastError(DrillClientError* pErr){
     if(pErr!=NULL){
-        std::map<int, DrillClientQueryResult*>::iterator iter;
-        if(!m_queryIds.empty()){
-            for(iter = m_queryIds.begin(); iter != m_queryIds.end(); iter++) {
+        std::map<int, DrillClientQueryHandle*>::const_iterator iter;
+        if(!m_queryHandles.empty()){
+            for(iter = m_queryHandles.begin(); iter != m_queryHandles.end(); iter++) {
                 DrillClientError* err=new DrillClientError(pErr->status, pErr->errnum, pErr->msg);
                 iter->second->signalError(err);
             }
         }
         delete pErr;
     }
-    // We have an error at the connection level. Cancel the heartbeat. 
-    // And close the connection 
+    // We have an error at the connection level. Cancel the heartbeat.
+    // And close the connection
     m_heartbeatTimer.cancel();
     m_pendingRequests=0;
     m_cv.notify_one();
@@ -1132,7 +1509,7 @@ void DrillClientImpl::broadcastError(DrillClientError* pErr){
 // The implementation is similar to handleQryError
 status_t DrillClientImpl::handleTerminatedQryState(
         status_t status,
-        std::string msg,
+        const std::string& msg,
         DrillClientQueryResult* pQueryResult){
     assert(pQueryResult!=NULL);
     if(status==QRY_COMPLETED){
@@ -1145,21 +1522,22 @@ status_t DrillClientImpl::handleTerminatedQryState(
     return status;
 }
 
-
-void DrillClientImpl::clearMapEntries(DrillClientQueryResult* pQueryResult){
-    std::map<int, DrillClientQueryResult*>::iterator iter;
+void DrillClientImpl::removeQueryHandle(DrillClientQueryHandle* pQueryHandle){
     boost::lock_guard<boost::mutex> lock(m_dcMutex);
-    if(!m_queryIds.empty()){
-        for(iter=m_queryIds.begin(); iter!=m_queryIds.end(); iter++) {
-            if(pQueryResult==(DrillClientQueryResult*)iter->second){
-                m_queryIds.erase(iter->first);
+    if(!m_queryHandles.empty()){
+        for(std::map<int, DrillClientQueryHandle*>::const_iterator iter=m_queryHandles.begin(); iter!=m_queryHandles.end(); iter++) {
+            if(pQueryHandle==(DrillClientQueryHandle*)iter->second){
+                m_queryHandles.erase(iter->first);
                 break;
             }
         }
     }
+}
+
+void DrillClientImpl::removeQueryResult(DrillClientQueryResult* pQueryResult){
+    boost::lock_guard<boost::mutex> lock(m_dcMutex);
     if(!m_queryResults.empty()){
-        std::map<exec::shared::QueryId*, DrillClientQueryResult*, compareQueryId>::iterator it;
-        for(it=m_queryResults.begin(); it!=m_queryResults.end(); it++) {
+        for(std::map<exec::shared::QueryId*, DrillClientQueryResult*, compareQueryId>::const_iterator it=m_queryResults.begin(); it!=m_queryResults.end(); it++) {
             if(pQueryResult==(DrillClientQueryResult*)it->second){
                 m_queryResults.erase(it->first);
                 break;
@@ -1168,19 +1546,19 @@ void DrillClientImpl::clearMapEntries(DrillClientQueryResult* pQueryResult){
     }
 }
 
-void DrillClientImpl::sendAck(InBoundRpcMessage& msg, bool isOk){
+void DrillClientImpl::sendAck(const rpc::InBoundRpcMessage& msg, bool isOk){
     exec::rpc::Ack ack;
     ack.set_ok(isOk);
-    OutBoundRpcMessage ack_msg(exec::rpc::RESPONSE, exec::user::ACK, msg.m_coord_id, &ack);
+    rpc::OutBoundRpcMessage ack_msg(exec::rpc::RESPONSE, exec::user::ACK, msg.m_coord_id, &ack);
     boost::lock_guard<boost::mutex> lock(m_dcMutex);
     sendSync(ack_msg);
     DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "ACK sent" << std::endl;)
 }
 
-void DrillClientImpl::sendCancel(exec::shared::QueryId* pQueryId){
+void DrillClientImpl::sendCancel(const exec::shared::QueryId* pQueryId){
     boost::lock_guard<boost::mutex> lock(m_dcMutex);
     uint64_t coordId = this->getNextCoordinationId();
-    OutBoundRpcMessage cancel_msg(exec::rpc::REQUEST, exec::user::CANCEL_QUERY, coordId, pQueryId);
+    rpc::OutBoundRpcMessage cancel_msg(exec::rpc::REQUEST, exec::user::CANCEL_QUERY, coordId, pQueryId);
     sendSync(cancel_msg);
     DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "CANCEL sent" << std::endl;)
 }
@@ -1193,6 +1571,14 @@ void DrillClientImpl::shutdownSocket(){
     DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Socket shutdown" << std::endl;)
 }
 
+meta::DrillMetadata* DrillClientImpl::getMetadata() {
+    return new meta::DrillMetadata(*this);
+}
+
+void DrillClientImpl::freeMetadata(meta::DrillMetadata* metadata) {
+    delete metadata;
+}
+
 // This COPIES the FieldMetadata definition for the record batch.  ColumnDefs held by this
 // class are used by the async callbacks.
 status_t DrillClientQueryResult::setupColumnDefs(exec::shared::QueryData* pQueryData) {
@@ -1254,7 +1640,7 @@ status_t DrillClientQueryResult::defaultQueryResultsListener(void* ctx,
     //ctx; // unused, we already have the this pointer
     DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Query result listener called" << std::endl;)
     //check if the query has been canceled. IF so then return FAILURE. Caller will send cancel to the server.
-    if(this->m_bCancel){
+    if(this->isCancelled()){
         if(b!=NULL) delete b;
         return QRY_FAILURE;
     }
@@ -1284,7 +1670,7 @@ RecordBatch*  DrillClientQueryResult::peekNext(){
     //if no more data, return NULL;
     if(!m_bIsQueryPending) return NULL;
     DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Synchronous read waiting for data." << std::endl;)
-    while(!this->m_bHasData && !m_bHasError && m_bIsQueryPending) {
+    while(!this->m_bHasData && !this->hasError() && m_bIsQueryPending) {
         this->m_cv.wait(cvLock);
     }
     // READ but not remove first element from queue
@@ -1305,7 +1691,7 @@ RecordBatch*  DrillClientQueryResult::getNext() {
     }
 
     DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Synchronous read waiting for data." << std::endl;)
-    while(!this->m_bHasData && !m_bHasError && m_bIsQueryPending){
+    while(!this->m_bHasData && !this->hasError() && m_bIsQueryPending){
         this->m_cv.wait(cvLock);
     }
     // remove first element from queue
@@ -1322,33 +1708,60 @@ void DrillClientQueryResult::waitForData() {
     boost::unique_lock<boost::mutex> cvLock(this->m_cvMutex);
     //if no more data, return NULL;
     if(!m_bIsQueryPending) return;
-    while(!this->m_bHasData && !m_bHasError && m_bIsQueryPending) {
+    while(!this->m_bHasData && !this->hasError() && m_bIsQueryPending) {
         this->m_cv.wait(cvLock);
     }
 }
 
-void DrillClientQueryResult::cancel() {
+template<typename Listener, typename Value>
+status_t DrillClientBaseHandle<Listener, Value>::notifyListener(Value v, DrillClientError* pErr){
+	return m_pApplicationListener(getApplicationContext(), v, pErr);
+}
+
+void DrillClientQueryHandle::cancel() {
     this->m_bCancel=true;
 }
 
-void DrillClientQueryResult::signalError(DrillClientError* pErr){
+void DrillClientQueryHandle::signalError(DrillClientError* pErr){
     // Ignore return values from the listener.
     if(pErr!=NULL){
         if(m_pError!=NULL){
             delete m_pError; m_pError=NULL;
         }
         m_pError=pErr;
-        pfnQueryResultsListener pResultsListener=this->m_pResultsListener;
-        if(pResultsListener!=NULL){
-            pResultsListener(this, NULL, pErr);
-        }else{
-            defaultQueryResultsListener(this, NULL, pErr);
-        }
+        // TODO should it be protected by m_cvMutex?
+        m_bHasError=true;
+    }
+    return;
+}
+
+template<typename Listener, typename Value>
+void DrillClientBaseHandle<Listener, Value>::signalError(DrillClientError* pErr){
+    DrillClientQueryHandle::signalError(pErr);
+    // Ignore return values from the listener.
+    if(pErr!=NULL){
+        this->notifyListener(NULL, pErr);
+    }
+}
+
+status_t DrillClientQueryResult::notifyListener(RecordBatch* batch, DrillClientError* pErr) {
+    pfnQueryResultsListener pResultsListener=getApplicationListener();
+    if(pResultsListener!=NULL){
+        return pResultsListener(this, batch, pErr);
+    }else{
+        return defaultQueryResultsListener(this, batch, pErr);
+    }
+}
+
+void DrillClientQueryResult::signalError(DrillClientError* pErr){
+    DrillClientQueryHandle::signalError(pErr);
+    // Ignore return values from the listener.
+    if(pErr!=NULL){
+        this->notifyListener(NULL, pErr);
         {
             boost::lock_guard<boost::mutex> cvLock(this->m_cvMutex);
             m_bIsQueryPending=false;
             m_bHasData=false;
-            m_bHasError=true;
         }
         //Signal the cv in case there is a client waiting for data already.
         m_cv.notify_one();
@@ -1357,24 +1770,27 @@ void DrillClientQueryResult::signalError(DrillClientError* pErr){
 }
 
 void DrillClientQueryResult::signalComplete(){
-    pfnQueryResultsListener pResultsListener=this->m_pResultsListener;
-    if(pResultsListener!=NULL){
-        pResultsListener(this, NULL, NULL);
-    }else{
-        defaultQueryResultsListener(this, NULL, NULL);
-    }
+    this->notifyListener(NULL, NULL);
     {
         boost::lock_guard<boost::mutex> cvLock(this->m_cvMutex);
-        m_bIsQueryPending=false;
         m_bIsQueryPending=!(this->m_recordBatches.empty()&&m_queryState==exec::shared::QueryResult_QueryState_COMPLETED);
-        m_bHasError=false;
+        resetError();
     }
     //Signal the cv in case there is a client waiting for data already.
     m_cv.notify_one();
     return;
 }
 
+void DrillClientQueryHandle::clearAndDestroy(){
+    //Tell the parent to remove this from its lists
+    m_client.removeQueryHandle(this);
+
+    if(m_pError!=NULL){
+        delete m_pError; m_pError=NULL;
+    }
+}
 void DrillClientQueryResult::clearAndDestroy(){
+    DrillClientQueryHandle::clearAndDestroy();
     //free memory allocated for FieldMetadata objects saved in m_columnDefs;
     if(!m_columnDefs->empty()){
         for(std::vector<Drill::FieldMetadata*>::iterator it = m_columnDefs->begin(); it != m_columnDefs->end(); ++it){
@@ -1385,15 +1801,16 @@ void DrillClientQueryResult::clearAndDestroy(){
     if(this->m_pQueryId!=NULL){
         DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Clearing state for Query Id - " << debugPrintQid(*this->m_pQueryId) << std::endl;)
     }
+
     //Tell the parent to remove this from its lists
-    m_pClient->clearMapEntries(this);
+    this->client().removeQueryResult(this);
 
     //clear query id map entries.
     if(this->m_pQueryId!=NULL){
         delete this->m_pQueryId; this->m_pQueryId=NULL;
     }
     if(!m_recordBatches.empty()){
-        // When multiple qwueries execute in parallel we sometimes get an empty record batch back from the server _after_
+        // When multiple queries execute in parallel we sometimes get an empty record batch back from the server _after_
         // the last chunk has been received. We eventually delete it.
         DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Freeing Record batch(es) left behind "<< std::endl;)
         RecordBatch* pR=NULL;
@@ -1403,11 +1820,32 @@ void DrillClientQueryResult::clearAndDestroy(){
             delete pR;
         }
     }
-    if(m_pError!=NULL){
-        delete m_pError; m_pError=NULL;
+}
+
+status_t DrillClientPrepareHandle::setupPreparedStatement(const exec::user::PreparedStatement& pstmt) {
+    // Get columns schema information
+    const ::google::protobuf::RepeatedPtrField< ::exec::user::ResultColumnMetadata>& columns = pstmt.columns();
+    for(::google::protobuf::RepeatedPtrField< ::exec::user::ResultColumnMetadata>::const_iterator it = columns.begin(); it != columns.end(); ++it) {
+        FieldMetadata* metadata = new FieldMetadata;
+        metadata->set(*it);
+        m_columnDefs->push_back(metadata);
     }
+
+    // Copy server handle
+    this->m_preparedStatementHandle.CopyFrom(pstmt.server_handle());
+    return QRY_SUCCESS;
 }
 
+void DrillClientPrepareHandle::clearAndDestroy(){
+    DrillClientQueryHandle::clearAndDestroy();
+    //free memory allocated for FieldMetadata objects saved in m_columnDefs;
+    if(!m_columnDefs->empty()){
+        for(std::vector<Drill::FieldMetadata*>::iterator it = m_columnDefs->begin(); it != m_columnDefs->end(); ++it){
+            delete *it;
+        }
+        m_columnDefs->clear();
+    }
+}
 
 connectionStatus_t PooledDrillClientImpl::connect(const char* connStr){
     connectionStatus_t stat = CONN_SUCCESS;
@@ -1418,9 +1856,9 @@ connectionStatus_t PooledDrillClientImpl::connect(const char* connStr){
     Utils::parseConnectStr(connStr, pathToDrill, protocol, hostPortStr);
     if(!strcmp(protocol.c_str(), "zk")){
         // Get a list of drillbits
-        ZookeeperImpl zook;
+        ZookeeperClient zook(pathToDrill);
         std::vector<std::string> drillbits;
-        int err = zook.getAllDrillbits(hostPortStr.c_str(), pathToDrill.c_str(), drillbits);
+        int err = zook.getAllDrillbits(hostPortStr, drillbits);
         if(!err){
             Utils::shuffle(drillbits);
             // The original shuffled order is maintained if we shuffle first and then add any missing elements
@@ -1432,15 +1870,17 @@ connectionStatus_t PooledDrillClientImpl::connect(const char* connStr){
                 m_lastConnection++;
                 nextIndex = (m_lastConnection)%(getDrillbitCount());
             }
+
             DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Pooled Connection"
                     << "(" << (void*)this << ")"
-                    << ": Current counter is: " 
+                    << ": Current counter is: "
                     << m_lastConnection << std::endl;)
-                err=zook.getEndPoint(m_drillbits, nextIndex, e);
+                err=zook.getEndPoint(m_drillbits[nextIndex], e);
             if(!err){
                 host=boost::lexical_cast<std::string>(e.address());
                 port=boost::lexical_cast<std::string>(e.user_port());
             }
+            DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Choosing drillbit <" << nextIndex  << ">. Selected " << e.DebugString() << std::endl;)
         }
         if(err){
             return handleConnError(CONN_ZOOKEEPER_ERROR, getMessage(ERR_CONN_ZOOKEEPER, zook.getError().c_str()));
@@ -1475,7 +1915,7 @@ connectionStatus_t PooledDrillClientImpl::validateHandshake(DrillUserProperties*
     connectionStatus_t stat=CONN_FAILURE;
     // Keep a copy of the user properties
     if(props!=NULL){
-        m_pUserProperties = new DrillUserProperties;
+        m_pUserProperties = boost::shared_ptr<DrillUserProperties>(new DrillUserProperties);
         for(size_t i=0; i<props->size(); i++){
             m_pUserProperties->setProperty(
                     props->keyAt(i),
@@ -1486,10 +1926,10 @@ connectionStatus_t PooledDrillClientImpl::validateHandshake(DrillUserProperties*
     DrillClientImpl* pDrillClientImpl = getOneConnection();
     if(pDrillClientImpl != NULL){
         DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Validating handshake: (Pooled) " << pDrillClientImpl->m_connectedHost << std::endl;)
-        stat=pDrillClientImpl->validateHandshake(m_pUserProperties);
+        stat = pDrillClientImpl->validateHandshake(m_pUserProperties.get());
     }
     else{
-        stat =  handleConnError(CONN_NOTCONNECTED, getMessage(ERR_CONN_NOCONN));
+        stat = handleConnError(CONN_NOTCONNECTED, getMessage(ERR_CONN_NOCONN));
     }
     return stat;
 }
@@ -1505,16 +1945,52 @@ DrillClientQueryResult* PooledDrillClientImpl::SubmitQuery(::exec::shared::Query
     return pDrillClientQueryResult;
 }
 
-void PooledDrillClientImpl::freeQueryResources(DrillClientQueryResult* pQryResult){
-    // Nothing to do. If this class ever keeps track of executing queries then it will need 
-    // to implement this call to free any query specific resources the pool might have 
+DrillClientPrepareHandle* PooledDrillClientImpl::PrepareQuery(const std::string& plan, pfnPreparedStatementListener listener, void* listenerCtx){
+    DrillClientPrepareHandle* pDrillClientPrepareHandle = NULL;
+    DrillClientImpl* pDrillClientImpl = NULL;
+    pDrillClientImpl = getOneConnection();
+    if(pDrillClientImpl != NULL){
+        pDrillClientPrepareHandle=pDrillClientImpl->PrepareQuery(plan,listener,listenerCtx);
+        m_queriesExecuted++;
+    }
+    return pDrillClientPrepareHandle;
+}
+
+DrillClientQueryResult* PooledDrillClientImpl::ExecuteQuery(const PreparedStatement& pstmt, pfnQueryResultsListener listener, void* listenerCtx){
+    DrillClientQueryResult* pDrillClientQueryResult = NULL;
+    DrillClientImpl* pDrillClientImpl = NULL;
+    pDrillClientImpl = getOneConnection();
+    if(pDrillClientImpl != NULL){
+        pDrillClientQueryResult=pDrillClientImpl->ExecuteQuery(pstmt, listener, listenerCtx);
+        m_queriesExecuted++;
+    }
+    return pDrillClientQueryResult;
+}
+
+void PooledDrillClientImpl::freeQueryResources(DrillClientQueryHandle* pQryHandle){
+    // If this class ever keeps track of executing queries then it will need
+    // to implement this call to free any query specific resources the pool might have
     // allocated
-    return;
+
+    pQryHandle->client().freeQueryResources(pQryHandle);
+}
+
+meta::DrillMetadata* PooledDrillClientImpl::getMetadata() {
+    meta::DrillMetadata* metadata = NULL;
+    DrillClientImpl* pDrillClientImpl = getOneConnection();
+    if (pDrillClientImpl != NULL) {
+        metadata = pDrillClientImpl->getMetadata();
+    }
+    return metadata;
+}
+
+void PooledDrillClientImpl::freeMetadata(meta::DrillMetadata* metadata) {
+    metadata->client().freeMetadata(metadata);
 }
 
 bool PooledDrillClientImpl::Active(){
     boost::lock_guard<boost::mutex> lock(m_poolMutex);
-    for(std::vector<DrillClientImpl*>::iterator it = m_clientConnections.begin(); it != m_clientConnections.end(); ++it){
+    for(std::vector<DrillClientImpl*>::const_iterator it = m_clientConnections.begin(); it != m_clientConnections.end(); ++it){
         if((*it)->Active()){
             return true;
         }
@@ -1529,7 +2005,7 @@ void PooledDrillClientImpl::Close() {
         delete *it;
     }
     m_clientConnections.clear();
-    if(m_pUserProperties!=NULL){ delete m_pUserProperties; m_pUserProperties=NULL;}
+    m_pUserProperties.reset();
     if(m_pError!=NULL){ delete m_pError; m_pError=NULL;}
     m_lastConnection=-1;
     m_queriesExecuted=0;
@@ -1592,7 +2068,7 @@ DrillClientImpl* PooledDrillClientImpl::getOneConnection(){
                 if((ret=connect(m_connectStr.c_str()))==CONN_SUCCESS){
                     boost::lock_guard<boost::mutex> lock(m_poolMutex);
                     pDrillClientImpl=m_clientConnections.back();
-                    ret=pDrillClientImpl->validateHandshake(m_pUserProperties);
+                    ret=pDrillClientImpl->validateHandshake(m_pUserProperties.get());
                     if(ret!=CONN_SUCCESS){
                         delete pDrillClientImpl; pDrillClientImpl=NULL;
                         m_clientConnections.erase(m_clientConnections.end());
@@ -1602,251 +2078,14 @@ DrillClientImpl* PooledDrillClientImpl::getOneConnection(){
             if(ret!=CONN_SUCCESS){
                 break;
             }
-        } // need a new connection 
+        } // need a new connection
     }// while
 
     if(pDrillClientImpl==NULL){
         connectionStatus_t status = CONN_NOTCONNECTED;
-        handleConnError(status, getMessage(status));
+        handleConnError(status, getMessage(ERR_CONN_NOCONN));
     }
     return pDrillClientImpl;
 }
 
-char ZookeeperImpl::s_drillRoot[]="/drill/";
-char ZookeeperImpl::s_defaultCluster[]="drillbits1";
-
-ZookeeperImpl::ZookeeperImpl(){
-    m_pDrillbits=new String_vector;
-    m_bConnecting=true;
-    memset(&m_id, 0, sizeof(m_id));
-}
-
-ZookeeperImpl::~ZookeeperImpl(){
-    delete m_pDrillbits;
-}
-
-ZooLogLevel ZookeeperImpl::getZkLogLevel(){
-    //typedef enum {ZOO_LOG_LEVEL_ERROR=1,
-    //    ZOO_LOG_LEVEL_WARN=2,
-    //    ZOO_LOG_LEVEL_INFO=3,
-    //    ZOO_LOG_LEVEL_DEBUG=4
-    //} ZooLogLevel;
-    switch(DrillClientConfig::getLogLevel()){
-        case LOG_TRACE:
-        case LOG_DEBUG:
-            return ZOO_LOG_LEVEL_DEBUG;
-        case LOG_INFO:
-            return ZOO_LOG_LEVEL_INFO;
-        case LOG_WARNING:
-            return ZOO_LOG_LEVEL_WARN;
-        case LOG_ERROR:
-        case LOG_FATAL:
-        default:
-            return ZOO_LOG_LEVEL_ERROR;
-    }
-    return ZOO_LOG_LEVEL_ERROR;
-}
-
-int ZookeeperImpl::getAllDrillbits(const char* connectStr, const char* pathToDrill, std::vector<std::string>& drillbits){
-    uint32_t waitTime=30000; // 10 seconds
-    zoo_set_debug_level(getZkLogLevel());
-    zoo_deterministic_conn_order(1); // enable deterministic order
-    struct String_vector* pDrillbits=NULL;
-    m_zh = zookeeper_init(connectStr, watcher, waitTime, 0, this, 0);
-    if(!m_zh) {
-        m_err = getMessage(ERR_CONN_ZKFAIL);
-        zookeeper_close(m_zh);
-        return -1;
-    }else{
-        m_err="";
-        //Wait for the completion handler to signal successful connection
-        boost::unique_lock<boost::mutex> bufferLock(this->m_cvMutex);
-        boost::system_time const timeout=boost::get_system_time()+ boost::posix_time::milliseconds(waitTime);
-        while(this->m_bConnecting) {
-            if(!this->m_cv.timed_wait(bufferLock, timeout)){
-                m_err = getMessage(ERR_CONN_ZKTIMOUT);
-                zookeeper_close(m_zh);
-                return -1;
-            }
-        }
-    }
-    if(m_state!=ZOO_CONNECTED_STATE){
-        zookeeper_close(m_zh);
-        return -1;
-    }
-    int rc = ZOK;
-    if(pathToDrill==NULL || strlen(pathToDrill)==0){
-        m_rootDir=s_drillRoot;
-        m_rootDir += s_defaultCluster;
-    }else{
-        m_rootDir=pathToDrill;
-    }
-
-    pDrillbits = new String_vector;
-    rc=zoo_get_children(m_zh, m_rootDir.c_str(), 0, pDrillbits);
-    if(rc!=ZOK){
-        delete pDrillbits;
-        m_err=getMessage(ERR_CONN_ZKERR, rc);
-        zookeeper_close(m_zh);
-        return -1;
-    }
-    if(pDrillbits && pDrillbits->count > 0){
-        DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Found " << pDrillbits->count << " drillbits in cluster (" 
-                << connectStr << "/" << pathToDrill
-                << ")." <<std::endl;)
-            for(int i=0; i<pDrillbits->count; i++){
-                drillbits.push_back(pDrillbits->data[i]);
-            }
-        for(int i=0; i<drillbits.size(); i++){
-            DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "\t Unshuffled Drillbit id: " << drillbits[i] << std::endl;)
-        }
-    }
-    delete pDrillbits;
-    return 0;
-}
-
-int ZookeeperImpl::getEndPoint(std::vector<std::string>& drillbits, size_t index, exec::DrillbitEndpoint& endpoint){
-    int rc = ZOK;
-    exec::DrillServiceInstance drillServiceInstance;
-    if( drillbits.size() >0){
-        // pick the drillbit at 'index'
-        const char * bit=drillbits[index].c_str();
-        std::string s;
-        s=m_rootDir +  std::string("/") + bit;
-        int buffer_len=MAX_CONNECT_STR;
-        char buffer[MAX_CONNECT_STR+1];
-        struct Stat stat;
-        buffer[MAX_CONNECT_STR]=0;
-        rc= zoo_get(m_zh, s.c_str(), 0, buffer,  &buffer_len, &stat);
-        if(rc!=ZOK){
-            m_err=getMessage(ERR_CONN_ZKDBITERR, rc);
-            zookeeper_close(m_zh);
-            return -1;
-        }
-        exec::DrillServiceInstance drillServiceInstance;
-        drillServiceInstance.ParseFromArray(buffer, buffer_len);
-        endpoint=drillServiceInstance.endpoint();
-        DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Choosing drillbit <" <<index << ">. Selected " << drillServiceInstance.DebugString() << std::endl;)
-    }else{
-
-        m_err=getMessage(ERR_CONN_ZKNODBIT);
-        zookeeper_close(m_zh);
-        return -1;
-    }
-    return 0;
-}
-
-// Deprecated
-int ZookeeperImpl::connectToZookeeper(const char* connectStr, const char* pathToDrill){
-    uint32_t waitTime=30000; // 10 seconds
-    zoo_set_debug_level(getZkLogLevel());
-    zoo_deterministic_conn_order(1); // enable deterministic order
-    m_zh = zookeeper_init(connectStr, watcher, waitTime, 0, this, 0);
-    if(!m_zh) {
-        m_err = getMessage(ERR_CONN_ZKFAIL);
-        return CONN_FAILURE;
-    }else{
-        m_err="";
-        //Wait for the completion handler to signal successful connection
-        boost::unique_lock<boost::mutex> bufferLock(this->m_cvMutex);
-        boost::system_time const timeout=boost::get_system_time()+ boost::posix_time::milliseconds(waitTime);
-        while(this->m_bConnecting) {
-            if(!this->m_cv.timed_wait(bufferLock, timeout)){
-                m_err = getMessage(ERR_CONN_ZKTIMOUT);
-                return CONN_FAILURE;
-            }
-        }
-    }
-    if(m_state!=ZOO_CONNECTED_STATE){
-        return CONN_FAILURE;
-    }
-    int rc = ZOK;
-    char rootDir[MAX_CONNECT_STR+1];
-    if(pathToDrill==NULL || strlen(pathToDrill)==0){
-        strcpy(rootDir, (char*)s_drillRoot);
-        strcat(rootDir, s_defaultCluster);
-    }else{
-        strncpy(rootDir, pathToDrill, MAX_CONNECT_STR); rootDir[MAX_CONNECT_STR]=0;
-    }
-    rc=zoo_get_children(m_zh, (char*)rootDir, 0, m_pDrillbits);
-    if(rc!=ZOK){
-        m_err=getMessage(ERR_CONN_ZKERR, rc);
-        zookeeper_close(m_zh);
-        return -1;
-    }
-
-    //Let's pick a random drillbit.
-    if(m_pDrillbits && m_pDrillbits->count >0){
-
-        std::vector<std::string> randomDrillbits;
-        for(int i=0; i<m_pDrillbits->count; i++){
-            randomDrillbits.push_back(m_pDrillbits->data[i]);
-        }
-        //Use the same random shuffle as the Java client instead of picking a drillbit at random.
-        //Gives much better randomization when the size of the cluster is small.
-        std::random_shuffle(randomDrillbits.begin(), randomDrillbits.end());
-        const char * bit=randomDrillbits[0].c_str();
-        std::string s;
-
-        s=rootDir +  std::string("/") + bit;
-        int buffer_len=MAX_CONNECT_STR;
-        char buffer[MAX_CONNECT_STR+1];
-        struct Stat stat;
-        buffer[MAX_CONNECT_STR]=0;
-        rc= zoo_get(m_zh, s.c_str(), 0, buffer,  &buffer_len, &stat);
-        if(rc!=ZOK){
-            m_err=getMessage(ERR_CONN_ZKDBITERR, rc);
-            zookeeper_close(m_zh);
-            return -1;
-        }
-        m_drillServiceInstance.ParseFromArray(buffer, buffer_len);
-    }else{
-        m_err=getMessage(ERR_CONN_ZKNODBIT);
-        zookeeper_close(m_zh);
-        return -1;
-    }
-    return 0;
-}
-
-void ZookeeperImpl::close(){
-    zookeeper_close(m_zh);
-}
-
-void ZookeeperImpl::watcher(zhandle_t *zzh, int type, int state, const char *path, void* context) {
-    //From cli.c
-
-    /* Be careful using zh here rather than zzh - as this may be mt code
-     * the client lib may call the watcher before zookeeper_init returns */
-
-    ZookeeperImpl* self=(ZookeeperImpl*)context;
-    self->m_state=state;
-    if (type == ZOO_SESSION_EVENT) {
-        if (state == ZOO_CONNECTED_STATE) {
-        } else if (state == ZOO_AUTH_FAILED_STATE) {
-            self->m_err= getMessage(ERR_CONN_ZKNOAUTH);
-            zookeeper_close(zzh);
-            self->m_zh=0;
-        } else if (state == ZOO_EXPIRED_SESSION_STATE) {
-            self->m_err= getMessage(ERR_CONN_ZKEXP);
-            zookeeper_close(zzh);
-            self->m_zh=0;
-        }
-    }
-    // signal the cond var
-    {
-        if (state == ZOO_CONNECTED_STATE){
-            DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Connected to Zookeeper." << std::endl;)
-        }
-        boost::lock_guard<boost::mutex> bufferLock(self->m_cvMutex);
-        self->m_bConnecting=false;
-    }
-    self->m_cv.notify_one();
-}
-
-void ZookeeperImpl:: debugPrint(){
-    if(m_zh!=NULL && m_state==ZOO_CONNECTED_STATE){
-        DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << m_drillServiceInstance.DebugStrin

<TRUNCATED>

[10/15] drill git commit: DRILL-4420: C++ API for metadata access and prepared statements

Posted by pa...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/166c4ce7/contrib/native/client/src/include/drill/drillClient.hpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/include/drill/drillClient.hpp b/contrib/native/client/src/include/drill/drillClient.hpp
index a74f4bd..5e59885 100644
--- a/contrib/native/client/src/include/drill/drillClient.hpp
+++ b/contrib/native/client/src/include/drill/drillClient.hpp
@@ -23,27 +23,9 @@
 #include <vector>
 #include <boost/thread.hpp>
 #include "drill/common.hpp"
+#include "drill/collections.hpp"
 #include "drill/protobuf/Types.pb.h"
 
-
-#if defined _WIN32 || defined __CYGWIN__
-  #ifdef DRILL_CLIENT_EXPORTS
-      #define DECLSPEC_DRILL_CLIENT __declspec(dllexport)
-  #else
-    #ifdef USE_STATIC_LIBDRILL
-      #define DECLSPEC_DRILL_CLIENT
-    #else
-      #define DECLSPEC_DRILL_CLIENT  __declspec(dllimport)
-    #endif
-  #endif
-#else
-  #if __GNUC__ >= 4
-    #define DECLSPEC_DRILL_CLIENT __attribute__ ((visibility ("default")))
-  #else
-    #define DECLSPEC_DRILL_CLIENT
-  #endif
-#endif
-
 namespace exec{
     namespace shared{
         class DrillPBError;
@@ -57,6 +39,7 @@ class  DrillClientImplBase;
 class  DrillClientImpl;
 class  DrillClientQueryResult;
 class  FieldMetadata;
+class  PreparedStatement;
 class  RecordBatch;
 class  SchemaDef;
 
@@ -188,6 +171,17 @@ typedef status_t (*pfnQueryResultsListener)(QueryHandle_t ctx, RecordBatch* b, D
  */
 typedef status_t (*pfnSchemaListener)(void* ctx, FieldDefPtr f, DrillClientError* err);
 
+/**
+ * The prepared statement creation listener
+ *
+ * This function is called when a prepared statement is created, or if an error occurs during the prepared statement creation.
+ * This callback is only invoked once. 
+ * @param[in] ctx the listener context provided to getColumns
+ * @param[in] pstmt the prepared statement handle, NULL in case of error
+ * @param[in] err an error object, NULL in case of success
+ */
+typedef status_t (*pfnPreparedStatementListener)(void* ctx, PreparedStatement* pstmt, DrillClientError* err);
+
 /*
  * A Record Iterator instance is returned by the SubmitQuery class. Calls block until some data
  * is available, or until all data has been returned.
@@ -244,6 +238,938 @@ class DECLSPEC_DRILL_CLIENT RecordIterator{
     // first record batch with this definition
 };
 
+namespace meta {
+  // Set of template functions to create bitmasks
+  template<typename T>
+  inline T
+  operator&(T __a, T __b)
+  { return T(static_cast<int>(__a) & static_cast<int>(__b)); }
+  template<typename T>
+  inline T
+  operator|(T __a, T __b)
+  { return T(static_cast<int>(__a) | static_cast<int>(__b)); }
+  template<typename T>
+  inline T
+  operator^(T __a, T __b)
+  { return T(static_cast<int>(__a) ^ static_cast<int>(__b)); }
+  template<typename T>
+  inline T&
+  operator|=(T& __a, T __b)
+  { return __a = __a | __b; }
+  template<typename T>
+  inline T&
+  operator&=(T& __a, T __b)
+  { return __a = __a & __b; }
+  template<typename T>
+  inline T&
+  operator^=(T& __a, T __b)
+  { return __a = __a ^ __b; }
+  template<typename T>
+  inline T
+  operator~(T __a)
+  { return T(~static_cast<int>(__a)); }
+
+  /*
+   * Internal type for Date/Time literals support
+   */
+  enum _DateTimeLiteralSupport {
+    _DL_NONE                      = 0,
+    _DL_DATE                      = 1 << 1L,
+    _DL_TIME                      = 1 << 2L,
+    _DL_TIMESTAMP                 = 1 << 3L,
+    _DL_INTERVAL_YEAR             = 1 << 4L,
+    _DL_INTERVAL_MONTH            = 1 << 5L,
+    _DL_INTERVAL_DAY              = 1 << 6L,
+    _DL_INTERVAL_HOUR             = 1 << 7L,
+    _DL_INTERVAL_MINUTE           = 1 << 8L,
+    _DL_INTERVAL_SECOND           = 1 << 9L,
+    _DL_INTERVAL_YEAR_TO_MONTH    = 1 << 10L,
+    _DL_INTERVAL_DAY_TO_HOUR      = 1 << 11L,
+    _DL_INTERVAL_DAY_TO_MINUTE    = 1 << 12L,
+    _DL_INTERVAL_DAY_TO_SECOND    = 1 << 13L,
+    _DL_INTERVAL_HOUR_TO_MINUTE   = 1 << 14L,
+    _DL_INTERVAL_HOUR_TO_SECOND   = 1 << 15L,
+    _DL_INTERVAL_MINUTE_TO_SECOND = 1 << 16L
+  };
+
+  template inline _DateTimeLiteralSupport operator&(_DateTimeLiteralSupport __a, _DateTimeLiteralSupport __b);
+  template inline _DateTimeLiteralSupport operator|(_DateTimeLiteralSupport __a, _DateTimeLiteralSupport __b);
+  template inline _DateTimeLiteralSupport operator^(_DateTimeLiteralSupport __a, _DateTimeLiteralSupport __b);
+
+  template inline _DateTimeLiteralSupport& operator&=(_DateTimeLiteralSupport& __a, _DateTimeLiteralSupport __b);
+  template inline _DateTimeLiteralSupport& operator|=(_DateTimeLiteralSupport& __a, _DateTimeLiteralSupport __b);
+  template inline _DateTimeLiteralSupport& operator^=(_DateTimeLiteralSupport& __a, _DateTimeLiteralSupport __b);
+
+  template inline _DateTimeLiteralSupport operator~(_DateTimeLiteralSupport __a);
+
+  /**
+   * Date time literal support flags
+   */
+  typedef _DateTimeLiteralSupport DateTimeLiteralSupport;
+
+  /** Does not support Date/Time literals */
+  static const DateTimeLiteralSupport DL_NONE = _DL_NONE;
+  /** Supports DATE literal */
+  static const DateTimeLiteralSupport DL_DATE = _DL_DATE;
+  /** Supports TIME literal */
+  static const DateTimeLiteralSupport DL_TIME = _DL_TIME;
+  /** Supports TIMESTAMP literal */
+  static const DateTimeLiteralSupport DL_TIMESTAMP = _DL_TIMESTAMP;
+  /** Supports INTERVAL YEAR literal */
+  static const DateTimeLiteralSupport DL_INTERVAL_YEAR = _DL_INTERVAL_YEAR;
+  /** Supports INTERVAL MONTH literal */
+  static const DateTimeLiteralSupport DL_INTERVAL_MONTH = _DL_INTERVAL_MONTH;
+  /** Supports INTERVAL DAY literal */
+  static const DateTimeLiteralSupport DL_INTERVAL_DAY = _DL_INTERVAL_DAY;
+  /** Supports INTERVAL HOUR literal */
+  static const DateTimeLiteralSupport DL_INTERVAL_HOUR = _DL_INTERVAL_HOUR;
+  /** Supports INTERVAL MINUTE literal */
+  static const DateTimeLiteralSupport DL_INTERVAL_MINUTE = _DL_INTERVAL_MINUTE;
+  /** Supports INTERVAL SECOND literal */
+  static const DateTimeLiteralSupport DL_INTERVAL_SECOND = _DL_INTERVAL_SECOND;
+  /** Supports INTERVAL YEAR TO MONTH literal */
+  static const DateTimeLiteralSupport DL_INTERVAL_YEAR_TO_MONTH = _DL_INTERVAL_YEAR_TO_MONTH;
+  /** Supports INTERVAL DAY TO HOUR literal */
+  static const DateTimeLiteralSupport DL_INTERVAL_DAY_TO_HOUR = _DL_INTERVAL_DAY_TO_HOUR;
+  /** Supports INTERVAL DAY TO MINUTE literal */
+  static const DateTimeLiteralSupport DL_INTERVAL_DAY_TO_MINUTE = _DL_INTERVAL_DAY_TO_MINUTE;
+  /** Supports INTERVAL DAY TO SECOND literal */
+  static const DateTimeLiteralSupport DL_INTERVAL_DAY_TO_SECOND = _DL_INTERVAL_DAY_TO_SECOND;
+  /** Supports INTERVAL HOUR TO MINUTE literal */
+  static const DateTimeLiteralSupport DL_INTERVAL_HOUR_TO_MINUTE = _DL_INTERVAL_HOUR_TO_MINUTE;
+  /** Supports INTERVAL HOUR TO SECOND literal */
+  static const DateTimeLiteralSupport DL_INTERVAL_HOUR_TO_SECOND = _DL_INTERVAL_HOUR_TO_SECOND;
+  /** Supports INTERVAL MINUTE TO SECOND literal */
+  static const DateTimeLiteralSupport DL_INTERVAL_MINUTE_TO_SECOND = _DL_INTERVAL_MINUTE_TO_SECOND;
+
+  /*
+   * Internal type for COLLATE support
+   */
+  enum _CollateSupport {
+      _C_NONE       = 0,
+      _C_GROUPBY    = 1 << 1L
+  };
+
+  template inline _CollateSupport operator&(_CollateSupport __a, _CollateSupport __b);
+  template inline _CollateSupport operator|(_CollateSupport __a, _CollateSupport __b);
+  template inline _CollateSupport operator^(_CollateSupport __a, _CollateSupport __b);
+
+  template inline _CollateSupport& operator&=(_CollateSupport& __a, _CollateSupport __b);
+  template inline _CollateSupport& operator|=(_CollateSupport& __a, _CollateSupport __b);
+  template inline _CollateSupport& operator^=(_CollateSupport& __a, _CollateSupport __b);
+
+  template inline _CollateSupport operator~(_CollateSupport __a);
+
+
+  /**
+   * COLLATE support flags
+   */
+  typedef _CollateSupport CollateSupport;
+  static const CollateSupport C_NONE = _C_NONE;       /**< COLLATE clauses are not supported */
+  static const CollateSupport C_GROUPBY = _C_GROUPBY; /**< a COLLATE clause can be added after each grouping column */
+
+  /**
+   * Correlation names support flags
+   */
+  enum CorrelationNamesSupport {
+    CN_NONE            = 1, /**< Correlation names are not supported */
+    CN_DIFFERENT_NAMES = 2, /**< Correlation names are supported, but names have to be different
+    							 from the tables they represent */
+    CN_ANY_NAMES       = 3  /**< Correlation names are supported with no restriction on names */
+  };
+
+  /**
+   * Group by support
+   */
+  enum GroupBySupport {
+      GB_NONE,         /**< Do not support GROUP BY */
+      GB_SELECT_ONLY,  /**< Only support GROUP BY clause with non aggregated columns in the select list */
+      GB_BEYOND_SELECT,/**< Support GROUP BY clauses with columns absent from the select list
+      	  	  	  	  	    if all the non-aggregated column from the select list are also added. */
+      GB_UNRELATED     /** Support GROUP BY clauses with columns absent from the select list */
+  };
+
+  /**
+   * Identified case support
+   */
+  enum IdentifierCase {
+      IC_STORES_LOWER,  /**< Mixed case unquoted SQL identifier are treated as
+	  	  	  	  	  	    case insensitive and stored in lower case */
+      IC_STORES_MIXED,  /**< Mixed case unquoted SQL identifier are treated as
+	  	  	  	  	  	    case insensitive and stored in mixed case */
+      IC_STORES_UPPER,  /**< Mixed case unquoted SQL identifier are treated as
+	  	  	  	  	  	    case insensitive and stored in upper case */
+      IC_SUPPORTS_MIXED /**< Mixed case unquoted SQL identifier are treated as
+	  	  	  	  	  	    case sensitive and stored in mixed case */
+  };
+
+  /**
+   * Null collation support
+   */
+  enum NullCollation {
+      NC_AT_START,/**< NULL values are sorted at the start regardless of the order*/
+      NC_AT_END,  /**< NULL values are sorted at the end regardless of the order*/
+      NC_HIGH,    /**< NULL is the highest value */
+      NC_LOW      /**< NULL is the lowest value */
+  };
+
+
+  /*
+   * Internal type for Outer join support flags
+   */
+  enum _OuterJoinSupport {
+      _OJ_NONE                  = 0,      //!< _OJ_NONE
+      _OJ_LEFT                  = 1 << 1L,//!< _OJ_LEFT
+      _OJ_RIGHT                 = 1 << 2L,//!< _OJ_RIGHT
+      _OJ_FULL                  = 1 << 3L,//!< _OJ_FULL
+      _OJ_NESTED                = 1 << 4L,//!< _OJ_NESTED
+      _OJ_NOT_ORDERED           = 1 << 5L,//!< _OJ_NOT_ORDERED
+      _OJ_INNER                 = 1 << 6L,//!< _OJ_INNER
+      _OJ_ALL_COMPARISON_OPS    = 1 << 7L //!< _OJ_ALL_COMPARISON_OPS
+  };
+
+  template inline _OuterJoinSupport operator&(_OuterJoinSupport __a, _OuterJoinSupport __b);
+  template inline _OuterJoinSupport operator|(_OuterJoinSupport __a, _OuterJoinSupport __b);
+  template inline _OuterJoinSupport operator^(_OuterJoinSupport __a, _OuterJoinSupport __b);
+
+  template inline _OuterJoinSupport& operator&=(_OuterJoinSupport& __a, _OuterJoinSupport __b);
+  template inline _OuterJoinSupport& operator|=(_OuterJoinSupport& __a, _OuterJoinSupport __b);
+  template inline _OuterJoinSupport& operator^=(_OuterJoinSupport& __a, _OuterJoinSupport __b);
+
+  template inline _OuterJoinSupport operator~(_OuterJoinSupport __a);
+
+  /**
+   * Outer join support flags
+   */
+  typedef _OuterJoinSupport OuterJoinSupport;
+  /** Outer join is not supported */
+  static const OuterJoinSupport OJ_NONE                 = _OJ_NONE;
+  /** Left outer join is supported */
+  static const OuterJoinSupport OJ_LEFT                 = _OJ_LEFT;
+  /** Right outer join is supported */
+  static const OuterJoinSupport OJ_RIGHT                = _OJ_RIGHT;
+  /** Full outer join is supported */
+  static const OuterJoinSupport OJ_FULL                 = _OJ_FULL;
+  /** Nested outer join is supported */
+  static const OuterJoinSupport OJ_NESTED               = _OJ_NESTED;
+  /**
+   * The columns names in the ON clause of a outer join don't have to share the same
+   * order as their respective table names in the OUTER JOIN clause
+   */
+  static const OuterJoinSupport OJ_NOT_ORDERED          = _OJ_NOT_ORDERED;
+  /**
+   * The inner table can also be used in an inner join
+   */
+  static const OuterJoinSupport OJ_INNER                = _OJ_INNER;
+  /**
+   * Any comparison operator in supported in the ON clause.
+   */
+  static const OuterJoinSupport OJ_ALL_COMPARISON_OPS   = _OJ_ALL_COMPARISON_OPS;
+
+  /**
+   * Quoted Identified case support
+   */
+  enum QuotedIdentifierCase {
+      QIC_STORES_LOWER,  /**< Mixed case quoted SQL identifier are treated as
+	  	  	  	  	  	    case insensitive and stored in lower case */
+      QIC_STORES_MIXED,  /**< Mixed case quoted SQL identifier are treated as
+	  	  	  	  	  	    case insensitive and stored in mixed case */
+      QIC_STORES_UPPER,  /**< Mixed case quoted SQL identifier are treated as
+	  	  	  	  	  	    case insensitive and stored in upper case */
+      QIC_SUPPORTS_MIXED /**< Mixed case quoted SQL identifier are treated as
+	  	  	  	  	  	    case sensitive and stored in mixed case */
+  };
+
+  /*
+   * Internal Subquery support flags type
+   */
+  enum _SubQuerySupport {
+      _SQ_NONE          = 0,
+      _SQ_CORRELATED    = 1 << 1L,
+      _SQ_IN_COMPARISON = 1 << 2L,
+      _SQ_IN_EXISTS     = 1 << 3L,
+      _SQ_IN_INSERT     = 1 << 4L,
+      _SQ_IN_QUANTIFIED = 1 << 5L
+  };
+
+  template inline _SubQuerySupport operator&(_SubQuerySupport __a, _SubQuerySupport __b);
+  template inline _SubQuerySupport operator|(_SubQuerySupport __a, _SubQuerySupport __b);
+  template inline _SubQuerySupport operator^(_SubQuerySupport __a, _SubQuerySupport __b);
+
+  template inline _SubQuerySupport& operator&=(_SubQuerySupport& __a, _SubQuerySupport __b);
+  template inline _SubQuerySupport& operator|=(_SubQuerySupport& __a, _SubQuerySupport __b);
+  template inline _SubQuerySupport& operator^=(_SubQuerySupport& __a, _SubQuerySupport __b);
+
+  template inline _SubQuerySupport operator~(_SubQuerySupport __a);
+
+  /**
+   * SubQuery support flags
+   */
+  typedef _SubQuerySupport SubQuerySupport;
+  /**
+   * Subqueries are not supported
+   */
+  static const SubQuerySupport SQ_NONE             = _SQ_NONE;
+  /** Correlated subqueries are supported */
+  static const SubQuerySupport SQ_CORRELATED       = _SQ_CORRELATED;
+  /** Subqueries in comparison expressions are supported */
+  static const SubQuerySupport SQ_IN_COMPARISON    = _SQ_IN_COMPARISON;
+  /** Subqueries in EXISTS expressions are supported */
+  static const SubQuerySupport SQ_IN_EXISTS        = _SQ_IN_EXISTS;
+  /** Subqueries in INSERT expressions are supported */
+  static const SubQuerySupport SQ_IN_INSERT        = _SQ_IN_INSERT;
+  /** Subqueries in quantified expressions are supported */
+  static const SubQuerySupport SQ_IN_QUANTIFIED    = _SQ_IN_QUANTIFIED;
+
+  /*
+   * Internal Union support flags type
+   */
+  enum _UnionSupport {
+      _U_NONE       = 0,      //!< _U_NONE
+      _U_UNION      = 1 << 1L,//!< _U_UNION
+      _U_UNION_ALL  = 1 << 2L //!< _U_UNION_ALL
+  };
+
+  template inline _UnionSupport operator&(_UnionSupport __a, _UnionSupport __b);
+  template inline _UnionSupport operator|(_UnionSupport __a, _UnionSupport __b);
+  template inline _UnionSupport operator^(_UnionSupport __a, _UnionSupport __b);
+
+  template inline _UnionSupport& operator&=(_UnionSupport& __a, _UnionSupport __b);
+  template inline _UnionSupport& operator|=(_UnionSupport& __a, _UnionSupport __b);
+  template inline _UnionSupport& operator^=(_UnionSupport& __a, _UnionSupport __b);
+
+  template inline _UnionSupport operator~(_UnionSupport __a);
+
+  /**
+   * Union support flags
+   */
+  typedef _UnionSupport UnionSupport;
+  /** Union is not supported */
+  static const UnionSupport U_NONE       = _U_NONE;
+  /** UNION is supported */
+  static const UnionSupport U_UNION      = _U_UNION;
+  /** UNION ALL is supported */
+  static const UnionSupport U_UNION_ALL  = _U_UNION_ALL;
+
+  class DECLSPEC_DRILL_CLIENT CatalogMetadata {
+      protected:
+      CatalogMetadata() {};
+      public:
+      virtual ~CatalogMetadata() {};
+
+      virtual bool               hasCatalogName() const = 0;
+      virtual const std::string& getCatalogName() const = 0;
+
+      virtual bool               hasDescription() const = 0;
+      virtual const std::string& getDescription() const = 0;
+
+      virtual bool               hasConnect() const = 0;
+      virtual const std::string& getConnect() const = 0;
+  };
+
+  class DECLSPEC_DRILL_CLIENT SchemaMetadata {
+      protected:
+      SchemaMetadata() {};
+
+      public:
+      virtual ~SchemaMetadata() {};
+
+      virtual bool               hasCatalogName() const = 0;
+      virtual const std::string& getCatalogName() const = 0;
+
+      virtual bool               hasSchemaName() const = 0;
+      virtual const std::string& getSchemaName() const = 0;
+
+      virtual bool               hasOwnerName() const = 0;
+      virtual const std::string& getOwner() const = 0;
+
+      virtual bool               hasType() const = 0;
+      virtual const std::string& getType() const = 0;
+
+      virtual bool               hasMutable() const = 0;
+      virtual const std::string& getMutable() const = 0;
+  };
+
+  class DECLSPEC_DRILL_CLIENT TableMetadata {
+      protected:
+      TableMetadata() {};
+
+      public:
+      virtual ~TableMetadata() {};
+
+      virtual bool               hasCatalogName() const = 0;
+      virtual const std::string& getCatalogName() const = 0;
+
+      virtual bool               hasSchemaName() const = 0;
+      virtual const std::string& getSchemaName() const = 0;
+
+      virtual bool               hasTableName() const = 0;
+      virtual const std::string& getTableName() const = 0;
+
+      virtual bool               hasType() const = 0;
+      virtual const std::string& getType() const = 0;
+  };
+
+  class DECLSPEC_DRILL_CLIENT ColumnMetadata {
+      protected:
+      ColumnMetadata() {};
+
+      public:
+      virtual ~ColumnMetadata() {};
+
+      virtual bool               hasCatalogName() const = 0;
+      virtual const std::string& getCatalogName() const = 0;
+
+      virtual bool               hasSchemaName() const = 0;
+      virtual const std::string& getSchemaName() const = 0;
+
+      virtual bool               hasTableName() const = 0;
+      virtual const std::string& getTableName() const = 0;
+
+      virtual bool               hasColumnName() const = 0;
+      virtual const std::string& getColumnName() const = 0;
+
+      virtual bool               hasOrdinalPosition() const = 0;
+      virtual std::size_t        getOrdinalPosition() const = 0;
+
+      virtual bool               hasDefaultValue() const = 0;
+      virtual const std::string& getDefaultValue() const = 0;
+
+      virtual bool               hasNullable() const = 0;
+      virtual bool               isNullable() const = 0;
+
+      virtual bool               hasDataType() const = 0;
+      virtual const std::string& getDataType() const = 0;
+
+      virtual bool               hasColumnSize() const = 0;
+      virtual std::size_t        getColumnSize() const = 0;
+
+      virtual bool               hasCharMaxLength() const = 0;
+      virtual std::size_t        getCharMaxLength() const = 0;
+
+      virtual bool               hasCharOctetLength() const = 0;
+      virtual std::size_t        getCharOctetLength() const = 0;
+
+      virtual bool               hasNumericPrecision() const = 0;
+      virtual int32_t            getNumericPrecision() const = 0;
+
+      virtual bool               hasNumericRadix() const = 0;
+      virtual int32_t            getNumericRadix() const = 0;
+
+      virtual bool               hasNumericScale() const = 0;
+      virtual int32_t            getNumericScale() const = 0;
+
+      virtual bool               hasIntervalType() const = 0;
+      virtual const std::string& getIntervalType() const = 0;
+
+      virtual bool               hasIntervalPrecision() const = 0;
+      virtual int32_t            getIntervalPrecision() const = 0;
+  };
+}
+
+class DECLSPEC_DRILL_CLIENT Metadata {
+  public:
+    virtual ~Metadata() {};
+
+    /**
+     * Returns the connector name
+     *
+     * @return the connector name
+     */
+    virtual const std::string& getConnectorName() const = 0;
+
+    /**
+     * Returns the connector version string
+     *
+     * @return the connector version string
+     */
+    virtual const std::string& getConnectorVersion() const = 0;
+
+    /**
+     * Returns the connector major version
+     *
+     * @return the connector major version
+     */
+    virtual uint32_t getConnectorMajorVersion() const = 0;
+
+    /**
+     * Returns the connector minor version
+     *
+     * @return the connector minor version
+     */
+    virtual uint32_t getConnectorMinorVersion() const = 0;
+
+    /**
+     * Returns the connector patch version
+     *
+     * @return the connector patch version
+     */
+    virtual uint32_t getConnectorPatchVersion() const = 0;
+
+    /**
+     * Returns the server name
+     *
+     * @return the server name
+     */
+    virtual const std::string& getServerName() const = 0;
+
+    /**
+     * Returns the server version string
+     *
+     * @return the server version string
+     */
+    virtual const std::string& getServerVersion() const = 0;
+
+    /**
+     * Returns the server major version
+     *
+     * @return the server major version
+     */
+    virtual uint32_t getServerMajorVersion() const = 0;
+
+    /**
+     * Returns the server minor version
+     *
+     * @return the server minor version
+     */
+    virtual uint32_t getServerMinorVersion() const = 0;
+
+    /**
+     * Returns the server patch version
+     *
+     * @return the server patch version
+     */
+    virtual uint32_t getServerPatchVersion() const = 0;
+
+    /**
+     * Callback function invoked by getCatalogs when receiving results
+     *
+     * This callback is only invoked once. 
+     * @param[in] ctx the listener context provided to getCatalogs
+     * @param[in] metadata the catalog metadata, or NULL in case of error
+     * @param[in] err an error object, NULL in case of success
+     */
+    typedef status_t (*pfnCatalogMetadataListener)(void* ctx, const DrillCollection<meta::CatalogMetadata>* metadata, DrillClientError* err);
+
+    /**
+     * Get a list of catalogPattern available to the current connection.
+     * Only catalogs matching the catalogPattern LIKE expression are returned.
+     *
+     * @param[in] catalogPattern a catalog pattern
+     * @param[in] listener a metadata listener
+     * @param[in] context to be passed to the listener
+     * @param[out] the query handle
+     */
+    virtual status_t getCatalogs(const std::string& catalogPattern, pfnCatalogMetadataListener listener, void* listenerCtx, QueryHandle_t* qHandle) = 0;
+
+    /**
+     * Callback function invoked by getSchemas when receiving results
+     *
+     * This callback is only invoked once. 
+     * @param[in] ctx the listener context provided to getSchemas
+     * @param[in] metadata the schema metadata, or NULL in case of error
+     * @param[in] err an error object, NULL in case of success
+     */
+    typedef status_t (*pfnSchemaMetadataListener)(void* ctx, const DrillCollection<meta::SchemaMetadata>* metadata, DrillClientError* err);
+
+    /**
+     * Get a list of schemas available to the current connection.
+     * Only schemas matching the catalogPattern and schemaPattern LIKE expressions are returned.
+     *
+     * @param[in] catalogPattern a catalog pattern
+     * @param[in] schemaPattern a schema pattern
+     * @param[in] listener a metadata query listener
+     * @param[in] context to be passed to the listener
+     * @param[out] the query handle
+     */
+    virtual status_t getSchemas(const std::string& catalogPattern, const std::string& schemaPattern, pfnSchemaMetadataListener listener, void* listenerCtx, QueryHandle_t* qHandle) = 0;
+
+    /**
+     * Callback function invoked by getTables when receiving results
+     *
+     * This callback is only invoked once. 
+     * @param[in] ctx the listener context provided to getTables
+     * @param[in] metadata the table metadata, or NULL in case of error
+     * @param[in] err an error object, NULL in case of success
+     */
+    typedef status_t (*pfnTableMetadataListener)(void* ctx, const DrillCollection<meta::TableMetadata>* metadata, DrillClientError* err);
+
+    /**
+     * Get a list of tables available to the current connection.
+     * Only tables matching the catalogPattern, schemaPattern and tablePattern LIKE expressions are returned.
+     *
+     * @param[in] catalogPattern a catalog pattern
+     * @param[in] schemaPattern a schema pattern
+     * @param[in] tablePattern a table pattern
+     * @param[in] tableTypes a list of table types to look for. Pass NULL to not filter
+     * @param[in] listener a metadata query listener
+     * @param[in] context to be passed to the listener
+     * @param[out] the query handle
+     */
+    virtual status_t getTables(const std::string& catalogPattern, const std::string& schemaPattern, const std::string& tablePattern, const std::vector<std::string>* tableTypes, 
+                               pfnTableMetadataListener listener, void* listenerCtx, QueryHandle_t* qHandle) = 0;
+
+    /**
+     * Callback function invoked by getColumns when receiving results
+     *
+     * This callback is only invoked once. 
+     * @param[in] ctx the listener context provided to getColumns
+     * @param[in] metadata the columns metadata, or NULL in case of error
+     * @param[in] err an error object, NULL in case of success
+     */
+    typedef status_t (*pfnColumnMetadataListener)(void* ctx, const DrillCollection<meta::ColumnMetadata>* metadata, DrillClientError* err);
+
+    /**
+     * Get a list of columns available to the current connection.
+     * Only columns matching the catalogPattern, schemaPattern, tablePattern and columnPattern LIKE expressions are returned.
+     *
+     * @param[in] catalogPattern a catalog pattern
+     * @param[in] schemaPattern a schema pattern
+     * @param[in] tablePattern a table pattern
+     * @param[in] columnPattern a colum name pattern
+     * @param[in] listener a metadata query listener
+     * @param[in] context to be passed to the listener
+     * @param[out] the query handle
+     */
+    virtual status_t getColumns(const std::string& catalogPattern, const std::string& schemaPattern, const std:: string& tablePattern, const std::string& columnPattern, pfnColumnMetadataListener listener, void* listenerCtx, QueryHandle_t* qHandle) = 0;
+
+    // Capabilities
+    /**
+     * Return if the current user can use all tables returned by the getTables method
+     *
+     * @result true if the user can select any table, false otherwise
+     */
+    virtual bool areAllTableSelectable() const = 0;
+
+    /**
+     * Return if the catalog name is at the start of a fully qualified table name
+     *
+     * @return true if the catalog name is at the start, false otherwise.
+     */
+    virtual bool isCatalogAtStart() const = 0;
+
+    /**
+     * Return the string used as a separator between the catalog and the table name
+     *
+     * @return the catalog separator
+     */
+    virtual const std::string& getCatalogSeparator() const = 0;
+
+    /**
+     * Return the term used by the server to designate a catalog
+     *
+     * @return the catalog term
+     */
+    virtual const std::string& getCatalogTerm() const = 0;
+
+    /**
+     * Return if the server supports column aliasing
+     *
+     * @return true if the server supports column aliasing, false otherwise
+     */
+    virtual bool isColumnAliasingSupported() const = 0;
+
+    /**
+     * Return if the result of a NULL and a non-NULL values concatenation is NULL
+     *
+     * @return true if the result is NULL, false otherwise
+     */
+    virtual bool isNullPlusNonNullNull() const = 0;
+
+    /**
+     * Return if the CONVERT function supports conversion for the given types
+     *
+     * @return true if the conversion is supported, false otherwise
+     */
+    virtual bool isConvertSupported(common::MinorType from, common::MinorType to) const = 0;
+
+    /**
+     * Return what kind of correlation name support the server provides
+     *
+     * @return the correlation name supported by the server
+     */
+    virtual meta::CorrelationNamesSupport getCorrelationNames() const = 0;
+
+    /**
+     * Returns if the connection to the server is read-only
+     *
+     * @return true if the connection is read-only, false otherwise
+     */
+    virtual bool isReadOnly() const = 0;
+
+    /**
+     * Return what kind of date time literals the server supports
+     *
+     * @return a bitmask of supported date/time literals
+     */
+    virtual meta::DateTimeLiteralSupport getDateTimeLiteralsSupport() const = 0;
+
+    /**
+     * Return what kind of COLLATE expressions are supported
+     */
+    virtual meta::CollateSupport getCollateSupport() const = 0;
+
+    /**
+     * Return what kind of GROUP BY support the server provides
+     *
+     * @return the group by support
+     */
+    virtual meta::GroupBySupport getGroupBySupport() const = 0;
+
+    /**
+     * Returns how unquoted identifier are stored
+     *
+     * @return the unquoted identifier storage policy
+     */
+    virtual meta::IdentifierCase getIdentifierCase() const = 0;
+
+    /**
+     * Returns the string used to quote SQL identifiers
+     *
+     * @return the quote string
+     */
+    virtual const std::string& getIdentifierQuoteString() const = 0;
+
+    /**
+     * Returns the list of SQL keywords supported by the database
+     *
+     * @return a list of keywords
+     */
+    virtual const std::vector<std::string>& getSQLKeywords() const = 0;
+
+    /**
+     * Returns if LIKE operator supports an escape clause
+     *
+     * @return true if escape claused are supported
+     */
+    virtual bool isLikeEscapeClauseSupported() const = 0;
+
+    /**
+     * Returns the maximum number of hexa characters supported for binary literals
+     *
+     * @return the length, 0 if unlimited or unknown
+     */
+    virtual std::size_t getMaxBinaryLiteralLength() const = 0;
+
+    /**
+     * Returns the maximum length of catalog names
+     *
+     * @return the length, 0 if unlimited or unknown
+     */
+    virtual std::size_t getMaxCatalogNameLength() const = 0;
+
+    /**
+     * Returns the maximum number of characters for string literals
+     *
+     * @return the length, 0 if unlimited or unknown
+     */
+    virtual std::size_t getMaxCharLiteralLength() const = 0;
+
+    /**
+     * Returns the maximum length of column names
+     *
+     * @return the length, 0 if unlimited or unknown
+     */
+    virtual std::size_t getMaxColumnNameLength() const = 0;
+
+    /**
+     * Returns the maximum number of columns in GROUP BY expressions
+     *
+     * @return the maximum number, 0 if unlimited or unknown
+     */
+    virtual std::size_t getMaxColumnsInGroupBy() const = 0;
+
+    /**
+     * Returns the maximum number of columns in ORDER BY expressions
+     *
+     * @return the maximum number, 0 if unlimited or unknown
+     */
+    virtual std::size_t getMaxColumnsInOrderBy() const = 0;
+
+    /**
+     * Returns the maximum number of columns in a SELECT list
+     *
+     * @return the maximum number, 0 if unlimited or unknown
+     */
+    virtual std::size_t getMaxColumnsInSelect() const = 0;
+
+    /**
+     * Returns the maximum length for cursor names
+     *
+     * @return the maximum length, 0 if unlimited or unknown
+     */
+    virtual std::size_t getMaxCursorNameLength() const = 0;
+
+    /**
+     * Returns the maximum logical size for LOB types
+     *
+     * @return the maximum size, 0 if unlimited or unknown
+     */
+    virtual std::size_t getMaxLogicalLobSize() const = 0;
+
+    /**
+     * Returns the maximum number of statements
+     *
+     * @return the maximum number, 0 if unlimited or unknown
+     */
+    virtual std::size_t getMaxStatements() const = 0;
+
+    /**
+     * Returns the maximum number of bytes for a single row
+     * @return the maximum size, 0 if unlimited or unknown
+     */
+    virtual std::size_t getMaxRowSize() const = 0;
+
+    /**
+     * Returns if BLOB types are included in the maximum row size
+     *
+     * @return true if BLOB are included
+     */
+    virtual bool isBlobIncludedInMaxRowSize() const = 0;
+
+    /**
+     * Returns the maximum length for schema names
+     * @return the maximum length, 0 if unlimited or unknown
+     */
+    virtual std::size_t getMaxSchemaNameLength() const = 0;
+
+    /**
+     * Returns the maximum length for statements
+     * @return the maximum length, 0 if unlimited or unknown
+     */
+    virtual std::size_t getMaxStatementLength() const = 0;
+
+    /**
+     * Returns the maximum length for table names
+     * @return the maximum length, 0 if unlimited or unknown
+     */
+    virtual std::size_t getMaxTableNameLength() const = 0;
+
+    /**
+     * Returns the maximum number of tables in a SELECT expression
+     * @return the maximum number, 0 if unlimited or unknown
+     */
+    virtual std::size_t getMaxTablesInSelect() const = 0;
+
+    /**
+     * Returns the maximum length for user names
+     * @return the maximum length, 0 if unlimited or unknown
+     */
+    virtual std::size_t getMaxUserNameLength() const = 0;
+
+    /**
+     * Returns how NULL are sorted
+     *
+     * @return the NULL collation policy
+     */
+    virtual meta::NullCollation getNullCollation() const = 0;
+
+    /**
+     * Returns the list of supported numeric functions
+     * @return a list of function names
+     */
+    virtual const std::vector<std::string>& getNumericFunctions() const = 0;
+
+    /**
+     * Returns how outer joins are supported
+     * @return outer join support (as flags)
+     */
+    virtual meta::OuterJoinSupport getOuterJoinSupport() const = 0;
+
+    /**
+     * Returns if columns not in the SELECT column lists can be used
+     * in the ORDER BY expression
+     *
+     * @return true if unrelated columns are supported in ORDER BY
+     */
+    virtual bool isUnrelatedColumnsInOrderBySupported() const = 0;
+
+    /**
+     * Returns how quoted identifier are stored
+     *
+     * @return the quoted identifier storage policy
+     */
+    virtual meta::QuotedIdentifierCase getQuotedIdentifierCase() const = 0;
+
+    /**
+     * Returns the term used to designate schemas
+     *
+     * @return the term
+     */
+    virtual const std::string& getSchemaTerm() const = 0;
+
+    /**
+     * Return the string for escaping patterns in metadata queries
+     *
+     * @return the characters for escaping, empty if not supported
+     */
+    virtual const std::string& getSearchEscapeString() const = 0;
+
+    /**
+     * Returns the list of extra characters that can be used in identifier names
+     *
+     * Extra characters are those characters beyond a-z, A-Z, 0-9 and '_' (underscore)
+     *
+     * @return a list of characters
+     */
+    virtual const std::string& getSpecialCharacters() const = 0;
+
+    /**
+     * Returns the list of supported string functions
+     *
+     * @return a list of function names
+     */
+    virtual const std::vector<std::string>& getStringFunctions() const = 0;
+
+    /**
+     * Returns how subqueries are supported
+     *
+     * @return the subqueries support (as flags)
+     */
+    virtual meta::SubQuerySupport getSubQuerySupport() const = 0;
+
+    /**
+     * Returns the list of supported system functions
+     *
+     * @return a list of function names
+     */
+    virtual const std::vector<std::string>& getSystemFunctions() const = 0;
+
+    /**
+     * Returns the term used to designate tables
+     *
+     * @return the term
+     */
+    virtual const std::string& getTableTerm() const = 0;
+
+    /**
+     * Returns the list of supported date/time functions
+     *
+     * @return a list of function names
+     */
+    virtual const std::vector<std::string>& getDateTimeFunctions() const = 0;
+
+    /**
+     * Returns if transactions are supported
+     * @return true if transactions are supported
+     */
+    virtual bool isTransactionSupported() const = 0;
+
+    /**
+     * Returns how unions are supported
+     *
+     * @return the union support (as flags)
+     */
+    virtual meta::UnionSupport getUnionSupport() const = 0;
+
+    /**
+     * Returns if SELECT FOR UPDATE expressions are supported
+     *
+     * @return true if SELECT FOR UPDATE is supported
+     */
+    virtual bool isSelectForUpdateSupported() const = 0;
+};
+
 class DECLSPEC_DRILL_CLIENT DrillClient{
     public:
         /*
@@ -273,7 +1199,7 @@ class DECLSPEC_DRILL_CLIENT DrillClient{
          */
         DEPRECATED connectionStatus_t connect(const char* connectStr, const char* defaultSchema=NULL);
 
-        /*  
+        /*
          * Connect the client to a Drillbit using connection string and a set of user properties.
          * The connection string format can be found in comments of
          * [DRILL-780](https://issues.apache.org/jira/browse/DRILL-780)
@@ -325,10 +1251,30 @@ class DECLSPEC_DRILL_CLIENT DrillClient{
 
         /*
          * Submit a query asynchronously and wait for results to be returned through an iterator that returns
-         * results synchronously. The client app needs to call delete on the iterator when done.
+         * results synchronously. The client app needs to call freeQueryIterator on the iterator when done.
          */
         RecordIterator* submitQuery(Drill::QueryType t, const std::string& plan, DrillClientError* err);
 
+        /**
+         * Prepare a query.
+         *
+         * @param[in] sql the query to prepare
+         * @param[in] listener a callback to be notified when the prepared statement is created, or if an error occured
+         * @param[in] user context to provide to the callback
+         * @param[out] a handle on the query
+         */
+        status_t prepareQuery(const std::string& sql, pfnPreparedStatementListener listener, void* listenerCtx, QueryHandle_t* qHandle);
+
+        /*
+         * Execute a prepared statement.
+         *
+         * @param[in] pstmt the prepared statement to execute
+         * @param[in] listener a callback to be notified when results have arrived, or if an error occured
+         * @param[in] user context to provide to the callback
+         * @param[out] a handle on the query
+         */
+        status_t executeQuery(const PreparedStatement& pstmt, pfnQueryResultsListener listener, void* listenerCtx, QueryHandle_t* qHandle);
+
         /*
          * The client application should call this function to wait for results if it has registered a
          * listener.
@@ -360,7 +1306,7 @@ class DECLSPEC_DRILL_CLIENT DrillClient{
          * Applications using the sync query submit method should call freeQueryIterator to free up resources
          * once the RecordIterator is no longer being processed.
          */
-        void freeQueryIterator(RecordIterator** pIter){ delete *pIter; *pIter=NULL;};
+        void freeQueryIterator(RecordIterator** pIter){ delete *pIter; *pIter=NULL;}
 
         /*
          * Applications using the async query submit method should call freeRecordBatch to free up resources
@@ -368,7 +1314,15 @@ class DECLSPEC_DRILL_CLIENT DrillClient{
          */
         void freeRecordBatch(RecordBatch* pRecordBatch);
 
+        /**
+         * Get access to the server metadata
+         */
+        Metadata* getMetadata();
 
+        /**
+         * Free resources associated with the metadata object
+         */
+        void freeMetadata(Metadata** metadata);
 
     private:
         static DrillClientInitializer s_init;

http://git-wip-us.apache.org/repos/asf/drill/blob/166c4ce7/contrib/native/client/src/include/drill/drillc.hpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/include/drill/drillc.hpp b/contrib/native/client/src/include/drill/drillc.hpp
index 3697ee8..c8593f5 100644
--- a/contrib/native/client/src/include/drill/drillc.hpp
+++ b/contrib/native/client/src/include/drill/drillc.hpp
@@ -21,6 +21,8 @@
 
 #include "drill/common.hpp"
 #include "drill/drillClient.hpp"
+#include "drill/fieldmeta.hpp"
+#include "drill/preparedStatement.hpp"
 #include "drill/recordBatch.hpp"
 #include "drill/protobuf/Types.pb.h"
 

http://git-wip-us.apache.org/repos/asf/drill/blob/166c4ce7/contrib/native/client/src/include/drill/fieldmeta.hpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/include/drill/fieldmeta.hpp b/contrib/native/client/src/include/drill/fieldmeta.hpp
new file mode 100644
index 0000000..40c9cca
--- /dev/null
+++ b/contrib/native/client/src/include/drill/fieldmeta.hpp
@@ -0,0 +1,122 @@
+/*
+ * 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.
+ */
+
+#ifndef FIELDMETA_H
+#define FIELDMETA_H
+
+#include "drill/common.hpp"
+#include "drill/protobuf/Types.pb.h"
+
+namespace exec{
+    namespace shared{
+        class SerializedField;
+    };
+    namespace user{
+        class ResultColumnMetadata;
+    };
+};
+
+
+namespace Drill {
+
+class DECLSPEC_DRILL_CLIENT FieldMetadata{
+    public:
+        enum ColumnSearchability { UNKNOWN_SEARCHABILITY = 0, NONE = 1, CHAR = 2, NUMBER = 3, ALL = 4 };
+        enum ColumnUpdatability { UNKNOWN_UPDATABILITY = 0, READ_ONLY = 1, WRITABLE = 2 };
+
+        FieldMetadata(){};
+        void set(const exec::shared::SerializedField& f);
+        void set(const exec::user::ResultColumnMetadata& m);
+        const std::string& getName() const{ return m_name;}
+        common::MinorType getMinorType() const{ return m_minorType;}
+        common::DataMode getDataMode() const{return m_dataMode;}
+        uint32_t getValueCount() const{return m_valueCount;}
+        uint32_t getScale() const{return m_scale;}
+        uint32_t getPrecision() const{return m_precision;}
+        uint32_t getBufferLength() const{return m_bufferLength;}
+        const std::string& getCatalogName() const{return m_catalogName;}
+        const std::string& getSchemaName() const{return m_schemaName;}
+        const std::string& getTableName() const{return m_tableName;}
+        const std::string& getLabel() const{return m_label;}
+        const std::string& getSQLType() const{return m_sqlType;}
+        bool isNullable() const{return m_nullable;}
+        bool isSigned() const{return m_signed;}
+        uint32_t getDisplaySize() const{return m_displaySize;}
+        bool isAliased() const{return m_aliased;}
+        ColumnSearchability getSearchability() const{return m_searchability;}
+        ColumnUpdatability getUpdatability() const{return m_updatability;}
+        bool isAutoIncremented() const{return m_autoIncremented;}
+        bool isCaseSensitive() const{return m_caseSensitive;}
+        bool isSortable() const{return m_sortable;}
+        bool isCurrency() const{return m_currency;}
+        void copy(Drill::FieldMetadata& f){
+            m_name=f.m_name;
+            m_minorType=f.m_minorType;
+            m_dataMode=f.m_dataMode;
+            m_valueCount=f.m_valueCount;
+            m_scale=f.m_scale;
+            m_precision=f.m_precision;
+            m_bufferLength=f.m_bufferLength;
+            m_catalogName=f.m_catalogName;
+            m_schemaName=f.m_schemaName;
+            m_tableName=f.m_tableName;
+            m_label=f.m_label;
+            m_sqlType=f.m_sqlType;
+            m_nullable=f.m_nullable;
+            m_signed=f.m_signed;
+            m_displaySize=f.m_displaySize;
+            m_aliased=f.m_aliased;
+            m_searchability=f.m_searchability;
+            m_updatability=f.m_updatability;
+            m_autoIncremented=f.m_autoIncremented;
+            m_caseSensitive=f.m_caseSensitive;
+            m_sortable=f.m_sortable;
+            m_currency=f.m_currency;
+            m_columnSize=f.m_columnSize;
+        }
+
+    private:
+        std::string m_name;
+        common::MinorType m_minorType;
+        common::DataMode m_dataMode;
+        uint32_t m_valueCount;
+        uint32_t m_scale;
+        uint32_t m_precision;
+        uint32_t m_bufferLength;
+        std::string m_catalogName;
+        std::string m_schemaName;
+        std::string m_tableName;
+        std::string m_label;
+        std::string m_sqlType;
+        bool m_nullable;
+        bool m_signed;
+        uint32_t m_displaySize;
+        bool m_aliased;
+        ColumnSearchability m_searchability;
+        ColumnUpdatability m_updatability;
+        bool m_autoIncremented;
+        bool m_caseSensitive;
+        bool m_sortable;
+        bool m_currency;
+        uint32_t m_columnSize;
+
+};
+} // namespace
+
+#endif
+

http://git-wip-us.apache.org/repos/asf/drill/blob/166c4ce7/contrib/native/client/src/include/drill/preparedStatement.hpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/include/drill/preparedStatement.hpp b/contrib/native/client/src/include/drill/preparedStatement.hpp
new file mode 100644
index 0000000..2a7d15a
--- /dev/null
+++ b/contrib/native/client/src/include/drill/preparedStatement.hpp
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef PREPAREDSTATEMENT_H
+#define PREPAREDSTATEMENT_H
+
+#include <cstddef>
+#include <string>
+#include <vector>
+
+namespace Drill {
+class DECLSPEC_DRILL_CLIENT PreparedStatement{
+public:
+	virtual std::size_t getNumFields() const = 0;
+	virtual const Drill::FieldMetadata& getFieldMetadata(std::size_t index) const = 0;
+
+	virtual ~PreparedStatement() {};
+};
+
+} // namespace Drill
+
+#endif // PREPAREDSTATEMENT_H
+

http://git-wip-us.apache.org/repos/asf/drill/blob/166c4ce7/contrib/native/client/src/include/drill/recordBatch.hpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/include/drill/recordBatch.hpp b/contrib/native/client/src/include/drill/recordBatch.hpp
index 12cbad4..8d1a0a3 100644
--- a/contrib/native/client/src/include/drill/recordBatch.hpp
+++ b/contrib/native/client/src/include/drill/recordBatch.hpp
@@ -647,7 +647,7 @@ template <class VALUEHOLDER_CLASS_TYPE, class VALUE_VECTOR_TYPE>
                     sstr<<"NULL";
                     strncpy(buf, sstr.str().c_str(), nChars);
                 }else{
-                    return m_pVector->getValueAt(index, buf, nChars);
+                    m_pVector->getValueAt(index, buf, nChars);
                 }
             }
 
@@ -786,39 +786,6 @@ typedef NullableValueVectorTyped<IntervalHolder, ValueVectorInterval>  NullableV
 typedef NullableValueVectorTyped<IntervalDayHolder, ValueVectorIntervalDay>  NullableValueVectorIntervalDay;
 typedef NullableValueVectorTyped<IntervalYearHolder, ValueVectorIntervalYear>  NullableValueVectorIntervalYear;
 
-class DECLSPEC_DRILL_CLIENT FieldMetadata{
-    public:
-
-        FieldMetadata(){};
-        void set(const exec::shared::SerializedField& f);
-        const std::string& getName() const{ return m_name;}
-        common::MinorType getMinorType() const{ return m_minorType;}
-        common::DataMode getDataMode() const{return m_dataMode;}
-        uint32_t getValueCount() const{return m_valueCount;}
-        uint32_t getScale() const{return m_scale;}
-        uint32_t getPrecision() const{return m_precision;}
-        uint32_t getBufferLength() const{return m_bufferLength;}
-        void copy(Drill::FieldMetadata& f){
-            m_name=f.m_name;
-            m_minorType=f.m_minorType;
-            m_dataMode=f.m_dataMode;
-            m_valueCount=f.m_valueCount;
-            m_scale=f.m_scale;
-            m_precision=f.m_precision;
-            m_bufferLength=f.m_bufferLength;
-        }
-
-    private:
-        //exec::shared::FieldMetadata* m_pFieldMetadata;
-        std::string m_name;
-        common::MinorType m_minorType;
-        common::DataMode m_dataMode;
-        uint32_t m_valueCount;
-        uint32_t m_scale;
-        uint32_t m_precision;
-        uint32_t m_bufferLength;
-};
-
 class FieldBatch{
     public:
         FieldBatch(const Drill::FieldMetadata& fmd, const ByteBuf_t data, size_t start, size_t length):

http://git-wip-us.apache.org/repos/asf/drill/blob/166c4ce7/contrib/native/client/src/test/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/test/CMakeLists.txt b/contrib/native/client/src/test/CMakeLists.txt
index 892b58c..523734b 100644
--- a/contrib/native/client/src/test/CMakeLists.txt
+++ b/contrib/native/client/src/test/CMakeLists.txt
@@ -18,6 +18,7 @@
 
 # Drill Client unit tests
 set (TESTS_SRC_FILES
+    ${CMAKE_CURRENT_SOURCE_DIR}/CollectionsTest.cpp
     ${CMAKE_CURRENT_SOURCE_DIR}/UtilsTest.cpp
     )
 

http://git-wip-us.apache.org/repos/asf/drill/blob/166c4ce7/contrib/native/client/src/test/CollectionsTest.cpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/test/CollectionsTest.cpp b/contrib/native/client/src/test/CollectionsTest.cpp
new file mode 100644
index 0000000..ebac941
--- /dev/null
+++ b/contrib/native/client/src/test/CollectionsTest.cpp
@@ -0,0 +1,215 @@
+/*
+ * 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.
+ */
+
+#include <string>
+#include <vector>
+
+#include <boost/assign.hpp>
+#include <boost/shared_ptr.hpp>
+
+#include <cppunit/TestFixture.h>
+#include <cppunit/extensions/HelperMacros.h>
+
+#include "drill/collections.hpp"
+#include "collectionsImpl.hpp"
+
+namespace {
+template<typename T, typename Iter>
+class DrillVectorIteratorImpl: public Drill::impl::DrillIteratorImpl<T> {
+public:
+	typedef DrillVectorIteratorImpl<T, Iter> type;
+	typedef Drill::impl::DrillIteratorImpl<T> supertype;
+
+	DrillVectorIteratorImpl(const Iter& it): m_it(it) {};
+
+	T& operator*() const { return m_it.operator *();}
+	T* operator->() const { return m_it.operator->(); }
+
+	operator typename Drill::impl::DrillIteratorImpl<const T>::iterator_ptr() const { return typename Drill::impl::DrillIteratorImpl<const T>::iterator_ptr(new DrillVectorIteratorImpl<const T, Iter>(m_it)); }
+
+	DrillVectorIteratorImpl& operator++() {
+		m_it++; return *this;
+	}
+
+	bool operator==(const supertype& x) const {
+		const type& other(dynamic_cast<const type&>(x));
+		return m_it == other.m_it;
+	}
+
+	bool operator!=(const supertype& x) const { return !(*this==x); }
+
+private:
+	Iter m_it;
+};
+
+template<typename T>
+class DrillVectorImpl: public Drill::impl::DrillCollectionImpl<T> {
+public:
+	typedef Drill::impl::DrillCollectionImpl<T> supertype;
+
+	typedef typename supertype::iterator_ptr iterator_ptr;
+	typedef typename supertype::const_iterator_ptr const_iterator_ptr;
+
+	DrillVectorImpl() {}
+	DrillVectorImpl(const std::vector<T>& v): m_vector(v) {};
+
+	iterator_ptr begin() { return iterator_ptr(new IteratorImpl(m_vector.begin()));}
+	const_iterator_ptr begin() const { return const_iterator_ptr(new ConstIteratorImpl(m_vector.begin()));}
+	iterator_ptr end() { return iterator_ptr(new IteratorImpl(m_vector.end()));}
+	const_iterator_ptr end() const { return const_iterator_ptr(new ConstIteratorImpl(m_vector.end()));}
+
+private:
+	typedef DrillVectorIteratorImpl<T, typename std::vector<T>::iterator> IteratorImpl;
+	typedef DrillVectorIteratorImpl<const T, typename std::vector<T>::const_iterator> ConstIteratorImpl;
+	std::vector<T> m_vector;
+};
+
+template<typename T>
+class DrillVector: public Drill::DrillCollection<T> {
+public:
+	DrillVector(const std::vector<T>& v): Drill::DrillCollection<T>(typename Drill::DrillCollection<T>::ImplPtr(new DrillVectorImpl<T>(v))) {}
+};
+
+
+struct SimpleInterface {
+	virtual ~SimpleInterface() {}
+	virtual const std::string& foo() const = 0;
+	virtual std::string bar() = 0;
+};
+
+class SimpleImplementation: public SimpleInterface {
+public:
+	SimpleImplementation(const std::string& foo, const std::string& bar): m_foo(foo), m_bar(bar) {}
+
+	const std::string& foo() const { return m_foo; }
+	std::string bar() { return m_bar; }
+
+private:
+	std::string m_foo;
+	std::string m_bar;
+};
+
+} // anonymous namespace
+
+class CollectionsTest: public CppUnit::TestFixture {
+public:
+	CollectionsTest() {}
+
+
+	CPPUNIT_TEST_SUITE( CollectionsTest );
+	CPPUNIT_TEST( testSimpleCollection );
+	CPPUNIT_TEST( testSimpleConstCollection );
+	CPPUNIT_TEST( testDrillVectorConstIterator );
+	CPPUNIT_TEST( testDrillVectorIterator );
+	CPPUNIT_TEST( testDrillVectorConstPointer );
+	CPPUNIT_TEST_SUITE_END();
+
+	void testSimpleCollection() {
+		// basic test/proof of concept for collections.hpp
+
+		std::vector<std::string> v = boost::assign::list_of("foo")("bar");
+
+		DrillVector<std::string> drillCollection(v);
+		std::vector<std::string> result;
+		for(DrillVector<std::string>::const_iterator it = drillCollection.begin(); it != drillCollection.end(); ++it) {
+			result.push_back(*it);
+		}
+
+		CPPUNIT_ASSERT(result == v);
+	}
+
+	void testSimpleConstCollection() {
+		std::vector<std::string> v = boost::assign::list_of("foo")("bar");
+
+		const DrillVector<std::string> drillCollection(v);
+		std::vector<std::string> result;
+		for(DrillVector<std::string>::const_iterator it = drillCollection.begin(); it != drillCollection.end(); ++it) {
+			result.push_back(*it);
+		}
+
+		CPPUNIT_ASSERT(result == v);
+	}
+
+	void testDrillVectorConstIterator() {
+		typedef Drill::DrillVector<SimpleInterface, SimpleImplementation> SimpleInterfaceVector;
+		SimpleInterfaceVector v;
+
+		v.push_back(SimpleImplementation("foo1", "bar1"));
+		v.push_back(SimpleImplementation("foo2", "bar2"));
+
+		std::vector<std::string> resultFoo;
+		SimpleInterfaceVector::const_iterator it(v.begin());
+		for(; it != v.end(); ++it) {
+			resultFoo.push_back(it->foo());
+			// const-correctness: The following line does not compile if uncommented!
+			// resultBar.push_back(it->bar());
+		}
+
+		std::vector<std::string> expectedFoo = boost::assign::list_of("foo1")("foo2");
+
+		CPPUNIT_ASSERT(resultFoo == expectedFoo);
+	}
+
+	void testDrillVectorIterator() {
+		typedef Drill::DrillVector<SimpleInterface, SimpleImplementation> SimpleInterfaceVector;
+		SimpleInterfaceVector v;
+
+		v.push_back(SimpleImplementation("foo1", "bar1"));
+		v.push_back(SimpleImplementation("foo2", "bar2"));
+
+		std::vector<std::string> resultFoo;
+		std::vector<std::string> resultBar;
+		SimpleInterfaceVector::iterator it;
+		for(it = v.begin(); it != v.end(); ++it) {
+			resultFoo.push_back(it->foo());
+			resultBar.push_back(it->bar());
+		}
+
+		std::vector<std::string> expectedFoo = boost::assign::list_of("foo1")("foo2");
+		std::vector<std::string> expectedBar = boost::assign::list_of("bar1")("bar2");
+
+		CPPUNIT_ASSERT(resultFoo == expectedFoo);
+		CPPUNIT_ASSERT(resultBar == expectedBar);
+	}
+
+	// Check some const-correctness issues
+	// by getting iterators of a const collection
+	void testDrillVectorConstPointer() {
+		typedef Drill::DrillVector<SimpleInterface, SimpleImplementation> SimpleInterfaceVector;
+		boost::shared_ptr<SimpleInterfaceVector> v(new SimpleInterfaceVector);
+
+		const SimpleInterfaceVector* vv(v.get());
+
+		v->push_back(SimpleImplementation("foo1", "bar1"));
+		v->push_back(SimpleImplementation("foo2", "bar2"));
+
+		std::vector<std::string> resultFoo;
+		std::vector<std::string> resultBar;
+		SimpleInterfaceVector::const_iterator it;
+		for(it = vv->begin(); it != vv->end(); ++it) {
+			resultFoo.push_back(it->foo());
+		}
+
+		std::vector<std::string> expectedFoo = boost::assign::list_of("foo1")("foo2");
+
+		CPPUNIT_ASSERT(resultFoo == expectedFoo);
+	}
+
+};
+
+CPPUNIT_TEST_SUITE_REGISTRATION( CollectionsTest );

http://git-wip-us.apache.org/repos/asf/drill/blob/166c4ce7/protocol/src/main/protobuf/User.proto
----------------------------------------------------------------------
diff --git a/protocol/src/main/protobuf/User.proto b/protocol/src/main/protobuf/User.proto
index f78ea2b..96d0477 100644
--- a/protocol/src/main/protobuf/User.proto
+++ b/protocol/src/main/protobuf/User.proto
@@ -361,7 +361,7 @@ message ResultColumnMetadata {
   optional ColumnSearchability searchability = 13;
 
   /*
-   * Defaults to READ_ONLU
+   * Defaults to READ_ONLY
    */
   optional ColumnUpdatability updatability = 14;
 


[03/15] drill git commit: DRILL-3243: Added CSG mods. Fixed field names. Removed old test files Added Parse_url and parse_query() functions Fix unit test

Posted by pa...@apache.org.
DRILL-3243: Added CSG mods. Fixed field names.
Removed old test files
Added Parse_url and parse_query() functions
Fix unit test

This closes #607


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

Branch: refs/heads/master
Commit: 4a82bc137865393f6f88b31805ad7c62cc9ca20b
Parents: 46c0f2a
Author: cgivre <cg...@gmail.com>
Authored: Wed Oct 5 23:29:26 2016 -0400
Committer: Parth Chandra <pa...@apache.org>
Committed: Tue Nov 1 10:42:22 2016 -0700

----------------------------------------------------------------------
 .../exec/expr/fn/impl/ParseQueryFunction.java   |  87 ++++
 .../exec/expr/fn/impl/ParseUrlFunction.java     | 153 ++++++
 .../exec/store/httpd/HttpdFormatPlugin.java     | 487 -------------------
 .../exec/store/httpd/HttpdLogFormatPlugin.java  |   3 +-
 .../drill/exec/store/httpd/HttpdLogRecord.java  |   2 +-
 .../drill/exec/store/httpd/HttpdParser.java     | 123 ++++-
 .../drill/exec/store/httpd/HttpdParserTest.java |  48 ++
 .../dfs/TestFormatPluginOptionExtractor.java    |   3 +
 .../drill/exec/store/httpd/HttpdParserTest.java |  48 --
 .../store/httpd/TestHttpdLogFormatPlugin.java   |  97 ----
 .../drill/exec/store/httpd/TestHttpdPlugin.java |  31 --
 11 files changed, 411 insertions(+), 671 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/4a82bc13/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/ParseQueryFunction.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/ParseQueryFunction.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/ParseQueryFunction.java
new file mode 100644
index 0000000..7dce1fc
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/ParseQueryFunction.java
@@ -0,0 +1,87 @@
+package org.apache.drill.exec.expr.fn.impl;
+
+//*
+
+import io.netty.buffer.DrillBuf;
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.NullableVarCharHolder;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter;
+
+import javax.inject.Inject;
+
+/* Copyright 2001-2004 The Apache Software Foundation.
+*
+* Licensed 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.
+*/
+
+@FunctionTemplate(
+        name="parse_query",
+        scope= FunctionTemplate.FunctionScope.SIMPLE,
+        nulls = FunctionTemplate.NullHandling.NULL_IF_NULL
+)
+
+public class ParseQueryFunction implements DrillSimpleFunc {
+
+    @Param
+    NullableVarCharHolder input;
+
+    @Output
+    BaseWriter.ComplexWriter outWriter;
+
+    @Inject
+    DrillBuf outBuffer;
+
+    public void setup() {
+    }
+
+    public void eval() {
+
+        org.apache.drill.exec.vector.complex.writer.BaseWriter.MapWriter queryMapWriter = outWriter.rootAsMap();
+
+        String queryString = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(input.start, input.end, input.buffer);
+
+        if( queryString.isEmpty() || queryString.equals("null")){
+            queryString = "";
+        }
+
+        String firstLetter = queryString.substring(0, 1);
+
+        //If the first character is a &, it doesn't split properly.  This checks to see if the first character is an & and if so, removes it.
+        if(firstLetter.equals("&")){
+            queryString = queryString.substring(1);
+        }
+
+        String[] arguments = queryString.split("&");
+
+        for (int i = 0; i < arguments.length; i++) {
+            String[] queryParts = arguments[i].split("=");
+
+            org.apache.drill.exec.expr.holders.VarCharHolder rowHolder = new org.apache.drill.exec.expr.holders.VarCharHolder();
+
+            byte[] rowStringBytes = queryParts[1].getBytes();
+
+            outBuffer.reallocIfNeeded(rowStringBytes.length);
+            outBuffer.setBytes(0, rowStringBytes);
+
+            rowHolder.start = 0;
+            rowHolder.end = rowStringBytes.length;
+            rowHolder.buffer = outBuffer;
+
+            queryMapWriter.varChar(queryParts[0]).write(rowHolder);
+
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/4a82bc13/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/ParseUrlFunction.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/ParseUrlFunction.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/ParseUrlFunction.java
new file mode 100644
index 0000000..fa339d4
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/ParseUrlFunction.java
@@ -0,0 +1,153 @@
+package org.apache.drill.exec.expr.fn.impl;
+
+/*
+ * Copyright 2001-2004 The Apache Software Foundation.
+ *
+ * Licensed 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 io.netty.buffer.DrillBuf;
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.VarCharHolder;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter;
+
+import javax.inject.Inject;
+
+@FunctionTemplate(
+        name="parse_url",
+        scope= FunctionTemplate.FunctionScope.SIMPLE,
+        nulls = FunctionTemplate.NullHandling.NULL_IF_NULL
+)
+
+public class ParseUrlFunction implements DrillSimpleFunc {
+
+    @Param VarCharHolder input;
+
+    @Output BaseWriter.ComplexWriter outWriter;
+
+    @Inject DrillBuf outBuffer;
+
+    public void setup() {}
+
+    public void eval() {
+
+        org.apache.drill.exec.vector.complex.writer.BaseWriter.MapWriter urlMapWriter = outWriter.rootAsMap();
+
+        String urlString = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(input.start, input.end, input.buffer);
+
+        try {
+            java.net.URL aURL = new java.net.URL(urlString);
+
+            String protocol = aURL.getProtocol();
+            String authority = aURL.getAuthority();
+            String host = aURL.getHost();
+            java.lang.Integer port = aURL.getPort();
+            String path = aURL.getPath();
+            String query = aURL.getQuery();
+            String filename = aURL.getFile();
+            String ref = aURL.getRef();
+
+            org.apache.drill.exec.expr.holders.VarCharHolder rowHolder = new org.apache.drill.exec.expr.holders.VarCharHolder();
+
+            byte[] rowStringBytes = protocol.getBytes();
+
+            outBuffer.reallocIfNeeded(rowStringBytes.length);
+            outBuffer.setBytes(0, rowStringBytes);
+
+            rowHolder.start = 0;
+            rowHolder.end = rowStringBytes.length;
+            rowHolder.buffer = outBuffer;
+
+            urlMapWriter.varChar("protocol").write(rowHolder);
+
+
+            byte[] authRowStringBytes = authority.getBytes();
+
+            outBuffer.reallocIfNeeded(authRowStringBytes.length);
+            outBuffer.setBytes(0, authRowStringBytes);
+
+            rowHolder.start = 0;
+            rowHolder.end = authRowStringBytes.length;
+            rowHolder.buffer = outBuffer;
+
+            urlMapWriter.varChar("authority").write(rowHolder);
+
+
+            byte[] hostRowStringBytes = host.getBytes();
+
+            outBuffer.reallocIfNeeded(hostRowStringBytes.length);
+            outBuffer.setBytes(0, hostRowStringBytes);
+
+            rowHolder.start = 0;
+            rowHolder.end = hostRowStringBytes.length;
+            rowHolder.buffer = outBuffer;
+
+            urlMapWriter.varChar("host").write(rowHolder);
+
+
+            byte[] pathRowStringBytes = path.getBytes();
+
+            outBuffer.reallocIfNeeded(pathRowStringBytes.length);
+            outBuffer.setBytes(0, pathRowStringBytes);
+
+            rowHolder.start = 0;
+            rowHolder.end = pathRowStringBytes.length;
+            rowHolder.buffer = outBuffer;
+
+            urlMapWriter.varChar("path").write(rowHolder);
+
+
+            byte[] queryRowStringBytes = query.getBytes();
+
+            outBuffer.reallocIfNeeded(queryRowStringBytes.length);
+            outBuffer.setBytes(0, queryRowStringBytes);
+
+            rowHolder.start = 0;
+            rowHolder.end = queryRowStringBytes.length;
+            rowHolder.buffer = outBuffer;
+
+            urlMapWriter.varChar("query").write(rowHolder);
+
+
+            byte[] filenameRowStringBytes = filename.getBytes();
+
+            outBuffer.reallocIfNeeded(filenameRowStringBytes.length);
+            outBuffer.setBytes(0, filenameRowStringBytes);
+
+            rowHolder.start = 0;
+            rowHolder.end = filenameRowStringBytes.length;
+            rowHolder.buffer = outBuffer;
+
+            urlMapWriter.varChar("filename").write(rowHolder);
+
+
+            byte[] refRowStringBytes = ref.getBytes();
+
+            outBuffer.reallocIfNeeded(refRowStringBytes.length);
+            outBuffer.setBytes(0, refRowStringBytes);
+
+            rowHolder.start = 0;
+            rowHolder.end = refRowStringBytes.length;
+            rowHolder.buffer = outBuffer;
+
+            urlMapWriter.varChar("ref").write(rowHolder);
+
+            org.apache.drill.exec.expr.holders.IntHolder intHolder = new org.apache.drill.exec.expr.holders.IntHolder();
+            intHolder.value = port;
+            urlMapWriter.integer("port").write(intHolder);
+        }
+        catch (Exception e ) {}
+    }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/4a82bc13/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdFormatPlugin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdFormatPlugin.java
deleted file mode 100644
index 7b8dc0e..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdFormatPlugin.java
+++ /dev/null
@@ -1,487 +0,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.
- */
-package org.apache.drill.exec.store.httpd;
-
-import io.netty.buffer.DrillBuf;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-
-import nl.basjes.parse.core.Parser;
-import nl.basjes.parse.core.exceptions.DissectionFailure;
-import nl.basjes.parse.core.exceptions.InvalidDissectorException;
-import nl.basjes.parse.core.exceptions.MissingDissectorsException;
-import nl.basjes.parse.httpdlog.ApacheHttpdLogFormatDissector;
-import nl.basjes.parse.httpdlog.dissectors.HttpFirstLineDissector;
-
-import org.apache.drill.common.exceptions.ExecutionSetupException;
-import org.apache.drill.common.exceptions.UserException;
-import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.common.logical.FormatPluginConfig;
-import org.apache.drill.common.logical.StoragePluginConfig;
-import org.apache.drill.exec.ExecConstants;
-import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.ops.OperatorContext;
-import org.apache.drill.exec.physical.impl.OutputMutator;
-import org.apache.drill.exec.server.DrillbitContext;
-import org.apache.drill.exec.store.AbstractRecordReader;
-import org.apache.drill.exec.store.RecordWriter;
-import org.apache.drill.exec.store.dfs.DrillFileSystem;
-import org.apache.drill.exec.store.dfs.easy.EasyFormatPlugin;
-import org.apache.drill.exec.store.dfs.easy.EasyWriter;
-import org.apache.drill.exec.store.dfs.easy.FileWork;
-import org.apache.drill.exec.vector.complex.impl.VectorContainerWriter;
-import org.apache.drill.exec.vector.complex.writer.BaseWriter.ComplexWriter;
-import org.apache.drill.exec.vector.complex.writer.BigIntWriter;
-import org.apache.drill.exec.vector.complex.writer.Float8Writer;
-import org.apache.drill.exec.vector.complex.writer.VarCharWriter;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.FileSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.LineRecordReader;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.mapred.TextInputFormat;
-
-import com.fasterxml.jackson.annotation.JsonTypeName;
-import com.google.common.base.Charsets;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-
-public class HttpdFormatPlugin extends EasyFormatPlugin<HttpdFormatPlugin.HttpdLogFormatConfig> {
-
-  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HttpdFormatPlugin.class);
-
-  private static final String DEFAULT_EXTENSION = "httpd";
-
-  public HttpdFormatPlugin(
-      String name,
-      DrillbitContext context,
-      Configuration fsConf,
-      StoragePluginConfig storageConfig,
-      HttpdLogFormatConfig formatConfig) {
-    super(name, context, fsConf, storageConfig, formatConfig, true, false, true, true,
-        Lists.newArrayList(DEFAULT_EXTENSION), DEFAULT_EXTENSION);
-  }
-
-  @JsonTypeName("httpd")
-  public static class HttpdLogFormatConfig implements FormatPluginConfig {
-    public String format;
-  }
-
-  private class RecordReader extends AbstractRecordReader {
-
-    private final DrillFileSystem fs;
-    private final FileWork work;
-    private final FragmentContext fragmentContext;
-
-    private ComplexWriter writer;
-    private Parser<ComplexWriterFacade> parser;
-    private LineRecordReader lineReader;
-    private LongWritable lineNumber;
-    private ComplexWriterFacade record;
-    private DrillBuf managedBuffer;
-
-    public RecordReader(FragmentContext context, DrillFileSystem fs, FileWork work) {
-      this.fs = fs;
-      this.work = work;
-      fragmentContext = context;
-      managedBuffer = context.getManagedBuffer();
-    }
-
-    @Override
-    public void setup(OperatorContext context, OutputMutator output) throws ExecutionSetupException {
-
-      try {
-
-        parser = new PartiallyDissectedParser<ComplexWriterFacade>(
-            ComplexWriterFacade.class,
-            HttpdFormatPlugin.this.getConfig().format);
-        writer = new VectorContainerWriter(output);
-        record = new ComplexWriterFacade(writer);
-        record.addAsParseTarget(parser);
-
-        final Path path = fs.makeQualified(new Path(work.getPath()));
-        FileSplit split = new FileSplit(path, work.getStart(), work.getLength(), new String[] { "" });
-        TextInputFormat inputFormat = new TextInputFormat();
-        JobConf job = new JobConf(fs.getConf());
-        job.setInt("io.file.buffer.size", fragmentContext.getConfig()
-            .getInt(ExecConstants.TEXT_LINE_READER_BUFFER_SIZE));
-        job.setInputFormat(inputFormat.getClass());
-        lineReader = (LineRecordReader) inputFormat.getRecordReader(split, job, Reporter.NULL);
-        lineNumber = lineReader.createKey();
-      } catch (Exception e) {
-        throw handleAndGenerate("Failure in creating record reader", e);
-      }
-
-
-    }
-
-    private DrillBuf buf(int size) {
-      if (managedBuffer.capacity() < size) {
-        managedBuffer = managedBuffer.reallocIfNeeded(size);
-      }
-      return managedBuffer;
-    }
-
-    protected RuntimeException handleAndGenerate(String s, Exception e) {
-      throw UserException.dataReadError(e)
-          .message(s + "\n%s", e.getMessage())
-          .addContext("Path", work.getPath())
-          .addContext("Split Start", work.getStart())
-          .addContext("Split Length", work.getLength())
-          .addContext("Local Line Number", lineNumber.get())
-          .build(logger);
-    }
-
-    @Override
-    public int next() {
-      try {
-        final Text currentText = lineReader.createValue();
-
-        writer.allocate();
-        writer.reset();
-        int recordCount = 0;
-
-        for (; recordCount < 4095 && lineReader.next(lineNumber, currentText); recordCount++) {
-          writer.setPosition(recordCount);
-          parser.parse(record, currentText.toString());
-        }
-
-        writer.setValueCount(recordCount);
-        return recordCount;
-      } catch (DissectionFailure | InvalidDissectorException | MissingDissectorsException | IOException e) {
-        throw handleAndGenerate("Failure while reading httpd log record.", e);
-      }
-    }
-
-    @Override
-    public void cleanup() {
-
-      try {
-        if (lineReader != null) {
-          lineReader.close();
-        }
-      } catch (IOException e) {
-        logger.warn("Failure while closing Httpd reader.", e);
-      }
-    }
-
-    /**
-     * Maps Httpd Log Libraries calls to Drills ComplexWriter interface.
-     */
-    public class ComplexWriterFacade {
-      private final ComplexWriter writer;
-      private final Map<String, VarCharWriter> stringWriters = Maps.newHashMap();
-      private final Map<String, BigIntWriter> longWriters = Maps.newHashMap();
-      private final Map<String, Float8Writer> doubleWriters = Maps.newHashMap();
-
-      private ComplexWriterFacade(ComplexWriter writer) {
-        this.writer = writer;
-      }
-
-      @SuppressWarnings("unused")
-      public void set(final String name, final String value) {
-        if (value != null) {
-          final byte[] stringBytes = value.getBytes(Charsets.UTF_8);
-          final DrillBuf stringBuffer = buf(stringBytes.length);
-          stringBuffer.clear();
-          stringBuffer.writeBytes(stringBytes);
-          final VarCharWriter writer = stringWriters.get(name);
-          if (writer != null) {
-            writer.writeVarChar(0, stringBytes.length, stringBuffer);
-          } else {
-            logger.warn("Dropped string.  Name: {}, Value: {}", name, value);
-          }
-        }
-      }
-
-      @SuppressWarnings("unused")
-      public void set(String name, Long value) {
-        if (value != null) {
-          longWriters.get(name).writeBigInt(value);
-        }
-      }
-
-      @SuppressWarnings("unused")
-      public void set(String name, Double value) {
-        if (value != null) {
-          doubleWriters.get(name).writeFloat8(value);
-        }
-      }
-
-      private void add(Parser<ComplexWriterFacade> parser, String path, VarCharWriter writer)
-          throws NoSuchMethodException,
-          SecurityException {
-        stringWriters.put(path, writer);
-        parser.addParseTarget(
-            ComplexWriterFacade.class.getMethod("set", new Class[] { String.class, String.class }),
-            path);
-      }
-
-      @SuppressWarnings("unused")
-      private void add(Parser<ComplexWriterFacade> parser, String path, Float8Writer writer)
-          throws NoSuchMethodException,
-          SecurityException {
-        doubleWriters.put(path, writer);
-        parser.addParseTarget(
-            ComplexWriterFacade.class.getMethod("set", new Class[] { String.class, Double.class }),
-            path);
-      }
-
-      private void add(Parser<ComplexWriterFacade> parser, String path, BigIntWriter writer)
-          throws NoSuchMethodException,
-          SecurityException {
-        longWriters.put(path, writer);
-        parser.addParseTarget(
-            ComplexWriterFacade.class.getMethod("set", new Class[] { String.class, Long.class }),
-            path);
-      }
-
-      public void addAsParseTarget(Parser<ComplexWriterFacade> parser) {
-        try {
-
-          for (final String path : parser.getPossiblePaths()) {
-            switch (path) {
-            case "IP:connection.client.ip":
-              add(parser, path, writer.rootAsMap().map("client").varChar("ip"));
-              break;
-            case "IP:connection.client.peerip":
-              add(parser, path, writer.rootAsMap().map("client").varChar("peer_ip"));
-              break;
-            case "IP:connection.server.ip":
-              add(parser, path, writer.rootAsMap().map("server").varChar("ip"));
-              break;
-            case "BYTES:response.body.bytes":
-              add(parser, path, writer.rootAsMap().map("response").bigInt("bytes"));
-              break;
-            case "BYTES:response.body.bytesclf":
-              add(parser, path, writer.rootAsMap().map("response").bigInt("bytes"));
-              break;
-            case "HTTP.COOKIE:request.cookies.":
-              add(parser, path, writer.rootAsMap().map("request").varChar("cookies"));
-              break;
-            case "MICROSECONDS:server.process.time":
-              add(parser, path, writer.rootAsMap().map("response").bigInt("process_time"));
-              break;
-            case "FILENAME:server.filename":
-              add(parser, path, writer.rootAsMap().map("response").varChar("filename"));
-              break;
-            case "IP:connection.client.host":
-              add(parser, path, writer.rootAsMap().map("client").varChar("host"));
-              break;
-            case "PROTOCOL:request.protocol":
-              add(parser, path, writer.rootAsMap().map("request").varChar("protocol"));
-              break;
-            case "HTTP.HEADER:request.header.":
-              add(parser, path, writer.rootAsMap().map("request").varChar("header"));
-              break;
-            case "NUMBER:connection.keepalivecount":
-              add(parser, path, writer.rootAsMap().map("client").bigInt("keepalivecount"));
-              break;
-            case "NUMBER:connection.client.logname":
-              add(parser, path, writer.rootAsMap().map("request").bigInt("logname"));
-              break;
-            case "STRING:request.errorlogid":
-              add(parser, path, writer.rootAsMap().map("request").varChar("errorlogid"));
-              break;
-            case "HTTP.METHOD:request.method":
-              add(parser, path, writer.rootAsMap().map("request").varChar("method"));
-              break;
-            case "PORT:request.server.port.canonical":
-              add(parser, path, writer.rootAsMap().map("server").bigInt("canonical_port"));
-              break;
-            case "PORT:connection.server.port.canonical":
-              add(parser, path, writer.rootAsMap().map("server").bigInt("canonical_port"));
-              break;
-            case "PORT:connection.client.port":
-              add(parser, path, writer.rootAsMap().map("client").bigInt("port"));
-              break;
-            case "NUBMER:connection.server.child.processid":
-              add(parser, path, writer.rootAsMap().map("server").bigInt("process_id"));
-              break;
-            case "NUMBER:connection.server.child.threadid":
-              add(parser, path, writer.rootAsMap().map("server").bigInt("thread_id"));
-              break;
-            case "STRING:connection.server.child.hexthreadid":
-              add(parser, path, writer.rootAsMap().map("connection").varChar("hex_thread_id"));
-              break;
-            case "HTTP.QUERYSTRING:request.querystring":
-              add(parser, path, writer.rootAsMap().map("").varChar(""));
-              break;
-            case "HTTP.FIRSTLINE:request.firstline":
-              add(parser, path, writer.rootAsMap().map("").varChar(""));
-              break;
-            case "STRING:request.handler":
-              add(parser, path, writer.rootAsMap().map("request").varChar("handler"));
-              break;
-            case "STRING:request.status.original":
-              add(parser, path, writer.rootAsMap().map("request").varChar("status_original"));
-              break;
-            case "STRING:request.status.last":
-              add(parser, path, writer.rootAsMap().map("request").varChar("status_last"));
-              break;
-            case "TIME.STAMP:request.receive.time":
-              add(parser, path, writer.rootAsMap().map("request").varChar("timestamp"));
-              break;
-            case "TIME.EPOCH:request.receive.time.begin.msec":
-              add(parser, path, writer.rootAsMap().map("request").bigInt("begin_msec"));
-              break;
-            case "TIME.EPOCH:request.receive.time.end.msec":
-              add(parser, path, writer.rootAsMap().map("request").bigInt("end_msec"));
-              break;
-            case "TIME.EPOCH.USEC:request.receive.time.begin.usec":
-              add(parser, path, writer.rootAsMap().map("request").bigInt("begin_usec"));
-              break;
-            case "TIME.EPOCH.USEC:request.receive.time.end.usec":
-              add(parser, path, writer.rootAsMap().map("request").bigInt("end_usec"));
-              break;
-            case "TIME.EPOCH:request.receive.time.begin.msec_frac":
-              add(parser, path, writer.rootAsMap().map("request").bigInt("begin_msec_frac"));
-              break;
-            case "TIME.EPOCH:request.receive.time.end.msec_frac":
-              add(parser, path, writer.rootAsMap().map("request").varChar("end_msec_frac"));
-              break;
-            case "TIME.EPOCH.USEC_FRAC:request.receive.time.begin.usec_frac":
-              add(parser, path, writer.rootAsMap().map("request").varChar("begin_usec_frac"));
-              break;
-            case "TIME.EPOCH.USEC_FRAC:request.receive.time.end.usec_frac":
-              add(parser, path, writer.rootAsMap().map("request").varChar("end_usec_frac"));
-              break;
-            case "SECONDS:response.server.processing.time":
-              add(parser, path, writer.rootAsMap().map("response").varChar("processing_time"));
-              break;
-            case "STRING:connection.client.user":
-              add(parser, path, writer.rootAsMap().map("client").varChar("user"));
-              break;
-            case "URI:request.urlpath":
-              add(parser, path, writer.rootAsMap().map("request").varChar("url"));
-              break;
-            case "STRING:connection.server.name.canonical":
-              add(parser, path, writer.rootAsMap().map("server").varChar("canonical_name"));
-              break;
-            case "STRING:connection.server.name":
-              add(parser, path, writer.rootAsMap().map("server").varChar("name"));
-              break;
-            case "HTTP.CONNECTSTATUS:response.connection.status":
-              add(parser, path, writer.rootAsMap().map("response").varChar("connection_status"));
-              break;
-            case "BYTES:request.bytes":
-              add(parser, path, writer.rootAsMap().map("request").varChar("bytes"));
-              break;
-            case "BYTES:response.bytes":
-              add(parser, path, writer.rootAsMap().map("response").bigInt("bytes"));
-              break;
-            case "HTTP.COOKIES:request.cookies":
-              add(parser, path, writer.rootAsMap().map("request").varChar("cookies"));
-              break;
-            case "HTTP.SETCOOKIES:response.cookies":
-              add(parser, path, writer.rootAsMap().map("response").varChar("cookies"));
-              break;
-            case "HTTP.USERAGENT:request.user-agent":
-              add(parser, path, writer.rootAsMap().map("request").varChar("useragent"));
-              break;
-            case "HTTP.URI:request.referer":
-              add(parser, path, writer.rootAsMap().map("request").varChar("referer"));
-              break;
-            case "HTTP.METHOD:method":
-              add(parser, path, writer.rootAsMap().map("request").varChar("method"));
-              break;
-            case "HTTP.URI:uri":
-              add(parser, path, writer.rootAsMap().map("request").varChar("uri"));
-              break;
-            case "HTTP.PROTOCOL:protocol":
-              add(parser, path, writer.rootAsMap().map("request").varChar("protocol"));
-              break;
-            case "HTTP.PROTOCOL.VERSION:protocol.version":
-              add(parser, path, writer.rootAsMap().map("request").varChar("protocol_version"));
-              break;
-            case "HTTP.METHOD:request.firstline.method":
-              add(parser, path, writer.rootAsMap().map("request").varChar("method"));
-              break;
-            case "HTTP.URI:request.firstline.uri":
-              add(parser, path, writer.rootAsMap().map("request").varChar("uri"));
-              break;
-            case "HTTP.PROTOCOL:request.firstline.protocol":
-              add(parser, path, writer.rootAsMap().map("request").varChar("protocol"));
-              break;
-            case "HTTP.PROTOCOL.VERSION:request.firstline.protocol.version":
-              add(parser, path, writer.rootAsMap().map("request").varChar("protocol_version"));
-              break;
-            default:
-
-              // if we don't know what to do, just write the raw value.
-              parser.addParseTarget(
-                  ComplexWriterFacade.class.getMethod("set", new Class[] { String.class, String.class }),
-                  path);
-              final String noPeriodPath = path.replace(".", "_");
-              stringWriters.put(path, writer.rootAsMap().varChar(noPeriodPath));
-              break;
-
-            }
-          }
-
-
-        } catch (MissingDissectorsException | SecurityException | NoSuchMethodException | InvalidDissectorException e) {
-          throw handleAndGenerate("Failure while setting up log mappings.", e);
-        }
-      }
-    }
-  }
-
-  @Override
-  public boolean supportsPushDown() {
-    return true;
-  }
-
-
-  @Override
-  public RecordReader getRecordReader(FragmentContext context, DrillFileSystem dfs,
-      FileWork fileWork, List<SchemaPath> columns) throws ExecutionSetupException {
-    return new RecordReader(context, dfs, fileWork);
-  }
-
-  @Override
-  public RecordWriter getRecordWriter(FragmentContext context, EasyWriter writer) throws IOException {
-    throw new UnsupportedOperationException("Drill doesn't currently support writing to HTTPD logs.");
-  }
-
-  @Override
-  public int getReaderOperatorType() {
-    return -1;
-  }
-
-  @Override
-  public int getWriterOperatorType() {
-    return -1;
-  }
-
-  private class PartiallyDissectedParser<RECORD> extends Parser<RECORD> {
-    public PartiallyDissectedParser(Class<RECORD> clazz, final String logformat) {
-      super(clazz);
-
-      addDissector(new ApacheHttpdLogFormatDissector(logformat));
-      addDissector(new HttpFirstLineDissector());
-      setRootType(ApacheHttpdLogFormatDissector.INPUT_TYPE);
-    }
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/4a82bc13/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdLogFormatPlugin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdLogFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdLogFormatPlugin.java
index 7a83d45..a6dc7d9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdLogFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdLogFormatPlugin.java
@@ -1,3 +1,4 @@
+
 /**
  * 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
@@ -243,4 +244,4 @@ public class HttpdLogFormatPlugin extends EasyFormatPlugin<HttpdLogFormatPlugin.
   public int getWriterOperatorType() {
     return -1;
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/4a82bc13/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdLogRecord.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdLogRecord.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdLogRecord.java
index 03f70c1..2775285 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdLogRecord.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdLogRecord.java
@@ -296,4 +296,4 @@ public class HttpdLogRecord {
       strings.put(parserFieldName, mapWriter.varChar(drillFieldName));
     }
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/4a82bc13/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdParser.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdParser.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdParser.java
index 114a7f4..a896638 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdParser.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdParser.java
@@ -18,9 +18,6 @@ package org.apache.drill.exec.store.httpd;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Maps;
 import io.netty.buffer.DrillBuf;
-import java.util.EnumSet;
-import java.util.List;
-import java.util.Map;
 import nl.basjes.parse.core.Casts;
 import nl.basjes.parse.core.Parser;
 import nl.basjes.parse.core.exceptions.DissectionFailure;
@@ -31,6 +28,11 @@ import org.apache.drill.exec.vector.complex.writer.BaseWriter.MapWriter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
 public class HttpdParser {
 
   private static final Logger LOG = LoggerFactory.getLogger(HttpdParser.class);
@@ -41,6 +43,105 @@ public class HttpdParser {
   private final Parser<HttpdLogRecord> parser;
   private final HttpdLogRecord record;
 
+    public static final HashMap<String, String> LOGFIELDS = new HashMap<String, String>();
+    static
+    {
+        LOGFIELDS.put("request_receive_time_weekyear__utc", "TIME_YEAR:request_receive_time_weekyear__utc");
+        LOGFIELDS.put("request_referer_ref", "HTTP_REF:request_referer_ref");
+        LOGFIELDS.put("request_referer_protocol", "HTTP_PROTOCOL:request_referer_protocol");
+        LOGFIELDS.put("request_receive_time_timezone", "TIME_ZONE:request_receive_time_timezone");
+        LOGFIELDS.put("connection_client_host", "IP:connection_client_host");
+        LOGFIELDS.put("connection_client_ip", "IP:connection_client_ip");
+        LOGFIELDS.put("connection_client_peerip", "IP:connection_client_peerip");
+        LOGFIELDS.put("connection_server_ip", "IP:connection_server_ip");
+        LOGFIELDS.put("request_receive_time_day", "TIME_DAY:request_receive_time_day");
+        LOGFIELDS.put("request_receive_time_minute__utc", "TIME_MINUTE:request_receive_time_minute__utc");
+        LOGFIELDS.put("request_referer_query_$", "STRING:request_referer_query_$");
+        LOGFIELDS.put("request_receive_time_millisecond__utc", "TIME_MILLISECOND:request_receive_time_millisecond__utc");
+        LOGFIELDS.put("request_firstline_uri_port", "HTTP_PORT:request_firstline_uri_port");
+        LOGFIELDS.put("request_referer_userinfo", "HTTP_USERINFO:request_referer_userinfo");
+        LOGFIELDS.put("request_receive_time_second__utc", "TIME_SECOND:request_receive_time_second__utc");
+        LOGFIELDS.put("request_firstline_uri_protocol", "HTTP_PROTOCOL:request_firstline_uri_protocol");
+        LOGFIELDS.put("request_receive_time_month", "TIME_MONTH:request_receive_time_month");
+        LOGFIELDS.put("request_firstline_uri_query", "HTTP_QUERYSTRING:request_firstline_uri_query");
+        LOGFIELDS.put("request_firstline_uri_path", "HTTP_PATH:request_firstline_uri_path");
+        LOGFIELDS.put("request_receive_time_hour__utc", "TIME_HOUR:request_receive_time_hour__utc");
+        LOGFIELDS.put("request_receive_time_monthname", "TIME_MONTHNAME:request_receive_time_monthname");
+        LOGFIELDS.put("request_receive_time_year__utc", "TIME_YEAR:request_receive_time_year__utc");
+        LOGFIELDS.put("request_receive_time_second", "TIME_SECOND:request_receive_time_second");
+        LOGFIELDS.put("request_referer", "HTTP_URI:request_referer");
+        LOGFIELDS.put("request_receive_time_monthname__utc", "TIME_MONTHNAME:request_receive_time_monthname__utc");
+        LOGFIELDS.put("request_referer_path", "HTTP_PATH:request_referer_path");
+        LOGFIELDS.put("request_receive_time_weekyear", "TIME_YEAR:request_receive_time_weekyear");
+        LOGFIELDS.put("request_firstline_protocol", "HTTP_PROTOCOL:request_firstline_protocol");
+        LOGFIELDS.put("request_referer_port", "HTTP_PORT:request_referer_port");
+        LOGFIELDS.put("request_receive_time_minute", "TIME_MINUTE:request_receive_time_minute");
+        LOGFIELDS.put("request_status_last", "STRING:request_status_last");
+        LOGFIELDS.put("request_receive_time_hour", "TIME_HOUR:request_receive_time_hour");
+        LOGFIELDS.put("request_firstline_protocol_version", "HTTP_PROTOCOL_VERSION:request_firstline_protocol_version");
+        LOGFIELDS.put("request_receive_time", "TIME_STAMP:request_receive_time");
+        LOGFIELDS.put("request_firstline_method", "HTTP_METHOD:request_firstline_method");
+        LOGFIELDS.put("request_receive_time_epoch", "TIME_EPOCH:request_receive_time_epoch");
+        LOGFIELDS.put("request_receive_time_weekofweekyear", "TIME_WEEK:request_receive_time_weekofweekyear");
+        LOGFIELDS.put("request_firstline_uri_host", "HTTP_HOST:request_firstline_uri_host");
+        LOGFIELDS.put("request_referer_query", "HTTP_QUERYSTRING:request_referer_query");
+        LOGFIELDS.put("request_firstline_uri_userinfo", "HTTP_USERINFO:request_firstline_uri_userinfo");
+        LOGFIELDS.put("response_body_bytes", "BYTES:response_body_bytes");
+        LOGFIELDS.put("response_body_bytesclf", "BYTES:response_body_bytesclf");
+        LOGFIELDS.put("request_referer_host", "HTTP_HOST:request_referer_host");
+        LOGFIELDS.put("request_receive_time_weekofweekyear__utc", "TIME_WEEK:request_receive_time_weekofweekyear__utc");
+        LOGFIELDS.put("request_firstline_uri", "HTTP_URI:request_firstline_uri");
+        LOGFIELDS.put("request_firstline_uri_ref", "HTTP_REF:request_firstline_uri_ref");
+        LOGFIELDS.put("request_receive_time_year", "TIME_YEAR:request_receive_time_year");
+        LOGFIELDS.put("request_firstline", "HTTP_FIRSTLINE:request_firstline");
+        LOGFIELDS.put("request_user-agent", "HTTP_USERAGENT:request_user-agent");
+        LOGFIELDS.put("request_cookies", "HTTP_COOKIE:request_cookies");
+        LOGFIELDS.put("server_process_time", "MICROSECONDS:server_process_time");
+        LOGFIELDS.put("request_cookies_$", "HTTP_COOKIE:request_cookies_$");
+        LOGFIELDS.put("server_environment_$", "VARIABLE:server_environment_$");
+        LOGFIELDS.put("server_filename", "FILENAME:server_filename");
+        LOGFIELDS.put("request_protocol", "PROTOCOL:request_protocol");
+        LOGFIELDS.put("request_header_", "HTTP_HEADER:request_header_");
+        LOGFIELDS.put("connection_keepalivecount", "NUMBER:connection_keepalivecount");
+        LOGFIELDS.put("connection_client_logname", "NUMBER:connection_client_logname");
+        LOGFIELDS.put("request_errorlogid", "STRING:request_errorlogid");
+        LOGFIELDS.put("request_method", "HTTP_METHOD:request_method");
+        LOGFIELDS.put("server_module_note_$", "STRING:server_module_note_$");
+        LOGFIELDS.put("response_header_$", "HTTP_HEADER:response_header_$");
+        LOGFIELDS.put("request_server_port_canonical", "PORT:request_server_port_canonical");
+        LOGFIELDS.put("connection_server_port_canonical", "PORT:connection_server_port_canonical");
+        LOGFIELDS.put("connection_server_port", "PORT:connection_server_port");
+        LOGFIELDS.put("connection_client_port", "PORT:connection_client_port");
+        LOGFIELDS.put("connection_server_child_processid", "NUMBER:connection_server_child_processid");
+        LOGFIELDS.put("connection_server_child_threadid", "NUMBER:connection_server_child_threadid");
+        LOGFIELDS.put("connection_server_child_hexthreadid", "NUMBER:connection_server_child_hexthreadid");
+        LOGFIELDS.put("request_querystring", "HTTP_QUERYSTRING:request_querystring");
+        LOGFIELDS.put("request_handler", "STRING:request_handler");
+        LOGFIELDS.put("request_status_original", "STRING:request_status_original");
+        LOGFIELDS.put("request_status_last", "STRING:request_status_last");
+        LOGFIELDS.put("request_receive_time_begin_msec", "TIME_EPOCH:request_receive_time_begin_msec");
+        LOGFIELDS.put("request_receive_time_end_msec", "TIME_EPOCH:request_receive_time_end_msec");
+        LOGFIELDS.put("request_receive_time_begin_usec", "TIME_EPOCH_USEC:request_receive_time_begin_usec");
+        LOGFIELDS.put("request_receive_time_begin_usec", "TIME_EPOCH_USEC:request_receive_time_begin_usec");
+        LOGFIELDS.put("request_receive_time_end_usec", "TIME_EPOCH_USEC:request_receive_time_end_usec");
+        LOGFIELDS.put("request_receive_time_begin_msec_frac", "TIME_EPOCH:request_receive_time_begin_msec_frac");
+        LOGFIELDS.put("request_receive_time_begin_msec_frac", "TIME_EPOCH:request_receive_time_begin_msec_frac");
+        LOGFIELDS.put("request_receive_time_end_msec_frac", "TIME_EPOCH:request_receive_time_end_msec_frac");
+        LOGFIELDS.put("request_receive_time_begin_usec_frac", "TIME_EPOCH_USEC_FRAC:request_receive_time_begin_usec_frac");
+        LOGFIELDS.put("request_receive_time_begin_usec_frac", "TIME_EPOCH_USEC_FRAC:request.receive.time.begin.usec_frac");
+        LOGFIELDS.put("request_receive_time_end_usec_frac", "TIME_EPOCH_USEC_FRAC:request_receive_time_end_usec_frac");
+        LOGFIELDS.put("response_server_processing_time", "SECONDS:response_server_processing_time");
+        LOGFIELDS.put("connection_client_user", "STRING:connection_client_user");
+        LOGFIELDS.put("request_urlpath", "URI:request_urlpath");
+        LOGFIELDS.put("connection_server_name_canonical", "STRING:connection_server_name_canonical");
+        LOGFIELDS.put("connection_server_name", "STRING:connection_server_name");
+        LOGFIELDS.put("response_connection_status", "HTTP_CONNECTSTATUS:response_connection_status");
+        LOGFIELDS.put("request_bytes", "BYTES:request_bytes");
+        LOGFIELDS.put("response_bytes", "BYTES:response_bytes");
+    }
+
+    //Map map = Collections.synchronizedMap(LOGFIELDS);
+
   public HttpdParser(final MapWriter mapWriter, final DrillBuf managedBuffer, final String logFormat,
       final String timestampFormat, final Map<String, String> fieldMapping)
       throws NoSuchMethodException, MissingDissectorsException, InvalidDissectorException {
@@ -96,7 +197,10 @@ public class HttpdParser {
    * @return
    */
   public static String parserFormattedFieldName(final String drillFieldName) {
-    return drillFieldName.replace(SAFE_WILDCARD, PARSER_WILDCARD).replaceAll(SAFE_SEPARATOR, ".").replaceAll("\\.\\.", "_");
+      String tempFieldName;
+      tempFieldName = LOGFIELDS.get(drillFieldName);
+      return tempFieldName.replace(SAFE_WILDCARD, PARSER_WILDCARD).replaceAll(SAFE_SEPARATOR, ".").replaceAll("\\.\\.", "_");
+
   }
 
   /**
@@ -108,7 +212,14 @@ public class HttpdParser {
    * @return
    */
   public static String drillFormattedFieldName(final String parserFieldName) {
-    return parserFieldName.replaceAll("_", "__").replace(PARSER_WILDCARD, SAFE_WILDCARD).replaceAll("\\.", SAFE_SEPARATOR);
+
+      if (parserFieldName.contains(":") ) {
+        String[] fieldPart= parserFieldName.split(":");
+        return fieldPart[1].replaceAll("_", "__").replace(PARSER_WILDCARD, SAFE_WILDCARD).replaceAll("\\.", SAFE_SEPARATOR);
+        }
+    else{
+      return parserFieldName.replaceAll("_", "__").replace(PARSER_WILDCARD, SAFE_WILDCARD).replaceAll("\\.", SAFE_SEPARATOR);
+    }
   }
 
   private void setupParser(final MapWriter mapWriter, final String logFormat, final Map<String, String> fieldMapping)
@@ -168,4 +279,4 @@ public class HttpdParser {
       record.addField(parser, mapWriter, casts, entry.getValue(), entry.getKey());
     }
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/4a82bc13/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdParserTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdParserTest.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdParserTest.java
new file mode 100644
index 0000000..b82b1ee
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/httpd/HttpdParserTest.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2015 The Apache Software Foundation.
+ *
+ * Licensed 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.drill.exec.store.httpd;
+
+import io.netty.buffer.DrillBuf;
+import java.util.Map;
+import org.apache.drill.exec.vector.complex.writer.BaseWriter.MapWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class HttpdParserTest {
+
+  private static final Logger LOG = LoggerFactory.getLogger(HttpdParserTest.class);
+
+  private void runTest(String logFormat, String logLine) throws Exception {
+    MapWriter mapWriter = null;
+    DrillBuf managedBuffer = null;
+    Map<String, String> configuredFields = null;
+    HttpdParser parser = new HttpdParser(mapWriter, managedBuffer, logFormat, null, configuredFields);
+    parser.parse(logLine);
+  }
+
+//  @Test
+  public void testFirstPattern() throws Exception {
+    LOG.info("testFirstPattern");
+//    final String format = "common";
+//    final String format = "%h %l %u %t \"%r\" %>s %b";
+    final String format = "%h %t \"%r\" %>s %b \"%{Referer}i\"";
+    final String line = "127.0.0.1 [31/Dec/2012:23:49:41 +0100] "
+        + "\"GET /foo HTTP/1.1\" 200 "
+        + "1213 \"http://localhost/index.php?mies=wim\"";
+    runTest(format, line);
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/4a82bc13/exec/java-exec/src/test/java/org/apache/drill/exec/store/dfs/TestFormatPluginOptionExtractor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/dfs/TestFormatPluginOptionExtractor.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/dfs/TestFormatPluginOptionExtractor.java
index c341295..2cbc09a 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/dfs/TestFormatPluginOptionExtractor.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/dfs/TestFormatPluginOptionExtractor.java
@@ -61,6 +61,9 @@ public class TestFormatPluginOptionExtractor {
         case "avro":
           assertEquals(d.typeName, "(type: String)", d.presentParams());
           break;
+        case "httpd":
+          assertEquals("(type: String, logFormat: String, timestampFormat: String)", d.presentParams());
+          break;
         default:
           fail("add validation for format plugin type " + d.typeName);
       }

http://git-wip-us.apache.org/repos/asf/drill/blob/4a82bc13/exec/java-exec/src/test/java/org/apache/drill/exec/store/httpd/HttpdParserTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/httpd/HttpdParserTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/httpd/HttpdParserTest.java
deleted file mode 100644
index 961d9a6..0000000
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/httpd/HttpdParserTest.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Copyright 2015 The Apache Software Foundation.
- *
- * Licensed 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.drill.exec.store.httpd;
-
-import io.netty.buffer.DrillBuf;
-import java.util.Map;
-import org.apache.drill.exec.vector.complex.writer.BaseWriter.MapWriter;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class HttpdParserTest {
-
-  private static final Logger LOG = LoggerFactory.getLogger(HttpdParserTest.class);
-
-  private void runTest(String logFormat, String logLine) throws Exception {
-    MapWriter mapWriter = null;
-    DrillBuf managedBuffer = null;
-    Map<String, String> configuredFields = null;
-    HttpdParser parser = new HttpdParser(mapWriter, managedBuffer, logFormat, null, configuredFields);
-    parser.parse(logLine);
-  }
-
-//  @Test
-  public void testFirstPattern() throws Exception {
-    LOG.info("testFirstPattern");
-//    final String format = "common";
-//    final String format = "%h %l %u %t \"%r\" %>s %b";
-    final String format = "%h %t \"%r\" %>s %b \"%{Referer}i\"";
-    final String line = "127.0.0.1 [31/Dec/2012:23:49:41 +0100] "
-        + "\"GET /foo HTTP/1.1\" 200 "
-        + "1213 \"http://localhost/index.php?mies=wim\"";
-    runTest(format, line);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/4a82bc13/exec/java-exec/src/test/java/org/apache/drill/exec/store/httpd/TestHttpdLogFormatPlugin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/httpd/TestHttpdLogFormatPlugin.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/httpd/TestHttpdLogFormatPlugin.java
deleted file mode 100644
index 6c2eaf8..0000000
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/httpd/TestHttpdLogFormatPlugin.java
+++ /dev/null
@@ -1,97 +0,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.
- */
-package org.apache.drill.exec.store.httpd;
-
-import java.util.List;
-import org.apache.drill.BaseTestQuery;
-import org.apache.drill.exec.rpc.user.QueryDataBatch;
-import static org.junit.Assert.*;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class TestHttpdLogFormatPlugin extends BaseTestQuery {
-
-  private static final Logger LOG = LoggerFactory.getLogger(TestHttpdLogFormatPlugin.class);
-
-  /**
-   * This test covers the test bootstrap-storage-plugins.json section of httpd.
-   *
-   * Indirectly this validates the HttpdLogFormatPlugin.HttpdLogFormatConfig deserializing properly.
-   *
-   * @throws Exception
-   */
-  @Test
-  public void testDfsTestBootstrap_star() throws Exception {
-    test("select * from dfs_test.`${WORKING_PATH}/src/test/resources/store/httpd/dfs-test-bootstrap-test.httpd`");
-  }
-
-  /**
-   * This test covers the test bootstrap-storage-plugins.json section of httpd.
-   *
-   * Indirectly this validates the HttpdLogFormatPlugin.HttpdLogFormatConfig deserializing properly.
-   *
-   * @throws Exception
-   */
-  @Test
-  public void testDfsTestBootstrap_notstar() throws Exception {
-    test("select `TIME_STAMP:request_receive_time`, `HTTP_METHOD:request_firstline_method`, `STRING:request_status_last`, `BYTES:response_body_bytesclf` \n"
-        + "from dfs_test.`${WORKING_PATH}/src/test/resources/store/httpd/dfs-test-bootstrap-test.httpd`");
-  }
-
-  /**
-   * This test covers the main bootstrap-storage-plugins.json section of httpd.
-   *
-   * @throws Exception
-   */
-  @Test
-  public void testDfsBootstrap_star() throws Exception {
-    test("select * from dfs.`${WORKING_PATH}/src/test/resources/store/httpd/dfs-bootstrap.httpd`");
-  }
-
-  /**
-   * This test covers the main bootstrap-storage-plugins.json section of httpd.
-   *
-   * @throws Exception
-   */
-  @Test
-  public void testDfsBootstrap_wildcard() throws Exception {
-    test("select `STRING:request_referer_query_$` from dfs.`${WORKING_PATH}/src/test/resources/store/httpd/dfs-bootstrap.httpd`");
-  }
-
-  /**
-   * This test covers the main bootstrap-storage-plugins.json section of httpd.
-   *
-   * @throws Exception
-   */
-  @Test
-  public void testDfsBootstrap_underscore() throws Exception {
-    test("select `TIME_DAY:request_receive_time_day__utc` from dfs.`${WORKING_PATH}/src/test/resources/store/httpd/dfs-bootstrap.httpd`");
-  }
-
-  @Test
-  public void testGroupBy_1() throws Exception {
-    final List<QueryDataBatch> actualResults = testSqlWithResults(
-        "select `HTTP_METHOD:request_firstline_method` as http_method, `STRING:request_status_last` as status_code, sum(`BYTES:response_body_bytesclf`) as total_bytes \n"
-        + "from dfs_test.`${WORKING_PATH}/src/test/resources/store/httpd/dfs-test-bootstrap-test.httpd`\n"
-        + "group by `HTTP_METHOD:request_firstline_method`, `STRING:request_status_last`"
-    );
-
-    final TestResultSet expectedResultSet = new TestResultSet();
-    expectedResultSet.addRow("GET", "200", "46551");
-    expectedResultSet.addRow("POST", "302", "18186");
-
-    TestResultSet actualResultSet = new TestResultSet(actualResults);
-    assertTrue(expectedResultSet.equals(actualResultSet));
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/4a82bc13/exec/java-exec/src/test/java/org/apache/drill/exec/store/httpd/TestHttpdPlugin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/httpd/TestHttpdPlugin.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/httpd/TestHttpdPlugin.java
deleted file mode 100644
index ce1f685..0000000
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/httpd/TestHttpdPlugin.java
+++ /dev/null
@@ -1,31 +0,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.
- */
-package org.apache.drill.exec.store.httpd;
-
-import org.apache.drill.BaseTestQuery;
-import org.junit.Test;
-
-public class TestHttpdPlugin extends BaseTestQuery {
-  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestHttpdPlugin.class);
-
-  @Test
-  public void tryBasicQuery() throws Exception {
-    // test("select * from cp.`store/httpd/example1.httpd`");
-    test("select * from dfs.`${WORKING_PATH}/src/test/resources/store/httpd/example1.httpd`");
-  }
-}


[08/15] drill git commit: DRILL-4853: Update C++ protobuf source files

Posted by pa...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/2558803e/contrib/native/client/src/protobuf/User.pb.cc
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/protobuf/User.pb.cc b/contrib/native/client/src/protobuf/User.pb.cc
index 212ad6a..4b0bf10 100644
--- a/contrib/native/client/src/protobuf/User.pb.cc
+++ b/contrib/native/client/src/protobuf/User.pb.cc
@@ -27,21 +27,87 @@ const ::google::protobuf::internal::GeneratedMessageReflection*
 const ::google::protobuf::Descriptor* UserProperties_descriptor_ = NULL;
 const ::google::protobuf::internal::GeneratedMessageReflection*
   UserProperties_reflection_ = NULL;
+const ::google::protobuf::Descriptor* RpcEndpointInfos_descriptor_ = NULL;
+const ::google::protobuf::internal::GeneratedMessageReflection*
+  RpcEndpointInfos_reflection_ = NULL;
 const ::google::protobuf::Descriptor* UserToBitHandshake_descriptor_ = NULL;
 const ::google::protobuf::internal::GeneratedMessageReflection*
   UserToBitHandshake_reflection_ = NULL;
 const ::google::protobuf::Descriptor* RequestResults_descriptor_ = NULL;
 const ::google::protobuf::internal::GeneratedMessageReflection*
   RequestResults_reflection_ = NULL;
-const ::google::protobuf::Descriptor* RunQuery_descriptor_ = NULL;
+const ::google::protobuf::Descriptor* GetQueryPlanFragments_descriptor_ = NULL;
 const ::google::protobuf::internal::GeneratedMessageReflection*
-  RunQuery_reflection_ = NULL;
+  GetQueryPlanFragments_reflection_ = NULL;
+const ::google::protobuf::Descriptor* QueryPlanFragments_descriptor_ = NULL;
+const ::google::protobuf::internal::GeneratedMessageReflection*
+  QueryPlanFragments_reflection_ = NULL;
 const ::google::protobuf::Descriptor* BitToUserHandshake_descriptor_ = NULL;
 const ::google::protobuf::internal::GeneratedMessageReflection*
   BitToUserHandshake_reflection_ = NULL;
+const ::google::protobuf::Descriptor* LikeFilter_descriptor_ = NULL;
+const ::google::protobuf::internal::GeneratedMessageReflection*
+  LikeFilter_reflection_ = NULL;
+const ::google::protobuf::Descriptor* GetCatalogsReq_descriptor_ = NULL;
+const ::google::protobuf::internal::GeneratedMessageReflection*
+  GetCatalogsReq_reflection_ = NULL;
+const ::google::protobuf::Descriptor* CatalogMetadata_descriptor_ = NULL;
+const ::google::protobuf::internal::GeneratedMessageReflection*
+  CatalogMetadata_reflection_ = NULL;
+const ::google::protobuf::Descriptor* GetCatalogsResp_descriptor_ = NULL;
+const ::google::protobuf::internal::GeneratedMessageReflection*
+  GetCatalogsResp_reflection_ = NULL;
+const ::google::protobuf::Descriptor* GetSchemasReq_descriptor_ = NULL;
+const ::google::protobuf::internal::GeneratedMessageReflection*
+  GetSchemasReq_reflection_ = NULL;
+const ::google::protobuf::Descriptor* SchemaMetadata_descriptor_ = NULL;
+const ::google::protobuf::internal::GeneratedMessageReflection*
+  SchemaMetadata_reflection_ = NULL;
+const ::google::protobuf::Descriptor* GetSchemasResp_descriptor_ = NULL;
+const ::google::protobuf::internal::GeneratedMessageReflection*
+  GetSchemasResp_reflection_ = NULL;
+const ::google::protobuf::Descriptor* GetTablesReq_descriptor_ = NULL;
+const ::google::protobuf::internal::GeneratedMessageReflection*
+  GetTablesReq_reflection_ = NULL;
+const ::google::protobuf::Descriptor* TableMetadata_descriptor_ = NULL;
+const ::google::protobuf::internal::GeneratedMessageReflection*
+  TableMetadata_reflection_ = NULL;
+const ::google::protobuf::Descriptor* GetTablesResp_descriptor_ = NULL;
+const ::google::protobuf::internal::GeneratedMessageReflection*
+  GetTablesResp_reflection_ = NULL;
+const ::google::protobuf::Descriptor* GetColumnsReq_descriptor_ = NULL;
+const ::google::protobuf::internal::GeneratedMessageReflection*
+  GetColumnsReq_reflection_ = NULL;
+const ::google::protobuf::Descriptor* ColumnMetadata_descriptor_ = NULL;
+const ::google::protobuf::internal::GeneratedMessageReflection*
+  ColumnMetadata_reflection_ = NULL;
+const ::google::protobuf::Descriptor* GetColumnsResp_descriptor_ = NULL;
+const ::google::protobuf::internal::GeneratedMessageReflection*
+  GetColumnsResp_reflection_ = NULL;
+const ::google::protobuf::Descriptor* CreatePreparedStatementReq_descriptor_ = NULL;
+const ::google::protobuf::internal::GeneratedMessageReflection*
+  CreatePreparedStatementReq_reflection_ = NULL;
+const ::google::protobuf::Descriptor* ResultColumnMetadata_descriptor_ = NULL;
+const ::google::protobuf::internal::GeneratedMessageReflection*
+  ResultColumnMetadata_reflection_ = NULL;
+const ::google::protobuf::Descriptor* PreparedStatementHandle_descriptor_ = NULL;
+const ::google::protobuf::internal::GeneratedMessageReflection*
+  PreparedStatementHandle_reflection_ = NULL;
+const ::google::protobuf::Descriptor* PreparedStatement_descriptor_ = NULL;
+const ::google::protobuf::internal::GeneratedMessageReflection*
+  PreparedStatement_reflection_ = NULL;
+const ::google::protobuf::Descriptor* CreatePreparedStatementResp_descriptor_ = NULL;
+const ::google::protobuf::internal::GeneratedMessageReflection*
+  CreatePreparedStatementResp_reflection_ = NULL;
+const ::google::protobuf::Descriptor* RunQuery_descriptor_ = NULL;
+const ::google::protobuf::internal::GeneratedMessageReflection*
+  RunQuery_reflection_ = NULL;
 const ::google::protobuf::EnumDescriptor* RpcType_descriptor_ = NULL;
 const ::google::protobuf::EnumDescriptor* QueryResultsMode_descriptor_ = NULL;
 const ::google::protobuf::EnumDescriptor* HandshakeStatus_descriptor_ = NULL;
+const ::google::protobuf::EnumDescriptor* RequestStatus_descriptor_ = NULL;
+const ::google::protobuf::EnumDescriptor* ColumnSearchability_descriptor_ = NULL;
+const ::google::protobuf::EnumDescriptor* ColumnUpdatability_descriptor_ = NULL;
 
 }  // namespace
 
@@ -83,8 +149,28 @@ void protobuf_AssignDesc_User_2eproto() {
       ::google::protobuf::DescriptorPool::generated_pool(),
       ::google::protobuf::MessageFactory::generated_factory(),
       sizeof(UserProperties));
-  UserToBitHandshake_descriptor_ = file->message_type(2);
-  static const int UserToBitHandshake_offsets_[7] = {
+  RpcEndpointInfos_descriptor_ = file->message_type(2);
+  static const int RpcEndpointInfos_offsets_[6] = {
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(RpcEndpointInfos, name_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(RpcEndpointInfos, version_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(RpcEndpointInfos, majorversion_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(RpcEndpointInfos, minorversion_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(RpcEndpointInfos, patchversion_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(RpcEndpointInfos, application_),
+  };
+  RpcEndpointInfos_reflection_ =
+    new ::google::protobuf::internal::GeneratedMessageReflection(
+      RpcEndpointInfos_descriptor_,
+      RpcEndpointInfos::default_instance_,
+      RpcEndpointInfos_offsets_,
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(RpcEndpointInfos, _has_bits_[0]),
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(RpcEndpointInfos, _unknown_fields_),
+      -1,
+      ::google::protobuf::DescriptorPool::generated_pool(),
+      ::google::protobuf::MessageFactory::generated_factory(),
+      sizeof(RpcEndpointInfos));
+  UserToBitHandshake_descriptor_ = file->message_type(3);
+  static const int UserToBitHandshake_offsets_[8] = {
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(UserToBitHandshake, channel_),
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(UserToBitHandshake, support_listening_),
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(UserToBitHandshake, rpc_version_),
@@ -92,6 +178,7 @@ void protobuf_AssignDesc_User_2eproto() {
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(UserToBitHandshake, properties_),
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(UserToBitHandshake, support_complex_types_),
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(UserToBitHandshake, support_timeout_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(UserToBitHandshake, client_infos_),
   };
   UserToBitHandshake_reflection_ =
     new ::google::protobuf::internal::GeneratedMessageReflection(
@@ -104,7 +191,7 @@ void protobuf_AssignDesc_User_2eproto() {
       ::google::protobuf::DescriptorPool::generated_pool(),
       ::google::protobuf::MessageFactory::generated_factory(),
       sizeof(UserToBitHandshake));
-  RequestResults_descriptor_ = file->message_type(3);
+  RequestResults_descriptor_ = file->message_type(4);
   static const int RequestResults_offsets_[2] = {
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(RequestResults, query_id_),
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(RequestResults, maximum_responses_),
@@ -120,29 +207,48 @@ void protobuf_AssignDesc_User_2eproto() {
       ::google::protobuf::DescriptorPool::generated_pool(),
       ::google::protobuf::MessageFactory::generated_factory(),
       sizeof(RequestResults));
-  RunQuery_descriptor_ = file->message_type(4);
-  static const int RunQuery_offsets_[3] = {
-    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(RunQuery, results_mode_),
-    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(RunQuery, type_),
-    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(RunQuery, plan_),
+  GetQueryPlanFragments_descriptor_ = file->message_type(5);
+  static const int GetQueryPlanFragments_offsets_[3] = {
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetQueryPlanFragments, query_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetQueryPlanFragments, type_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetQueryPlanFragments, split_plan_),
   };
-  RunQuery_reflection_ =
+  GetQueryPlanFragments_reflection_ =
     new ::google::protobuf::internal::GeneratedMessageReflection(
-      RunQuery_descriptor_,
-      RunQuery::default_instance_,
-      RunQuery_offsets_,
-      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(RunQuery, _has_bits_[0]),
-      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(RunQuery, _unknown_fields_),
+      GetQueryPlanFragments_descriptor_,
+      GetQueryPlanFragments::default_instance_,
+      GetQueryPlanFragments_offsets_,
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetQueryPlanFragments, _has_bits_[0]),
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetQueryPlanFragments, _unknown_fields_),
       -1,
       ::google::protobuf::DescriptorPool::generated_pool(),
       ::google::protobuf::MessageFactory::generated_factory(),
-      sizeof(RunQuery));
-  BitToUserHandshake_descriptor_ = file->message_type(5);
-  static const int BitToUserHandshake_offsets_[4] = {
+      sizeof(GetQueryPlanFragments));
+  QueryPlanFragments_descriptor_ = file->message_type(6);
+  static const int QueryPlanFragments_offsets_[4] = {
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(QueryPlanFragments, status_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(QueryPlanFragments, query_id_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(QueryPlanFragments, fragments_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(QueryPlanFragments, error_),
+  };
+  QueryPlanFragments_reflection_ =
+    new ::google::protobuf::internal::GeneratedMessageReflection(
+      QueryPlanFragments_descriptor_,
+      QueryPlanFragments::default_instance_,
+      QueryPlanFragments_offsets_,
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(QueryPlanFragments, _has_bits_[0]),
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(QueryPlanFragments, _unknown_fields_),
+      -1,
+      ::google::protobuf::DescriptorPool::generated_pool(),
+      ::google::protobuf::MessageFactory::generated_factory(),
+      sizeof(QueryPlanFragments));
+  BitToUserHandshake_descriptor_ = file->message_type(7);
+  static const int BitToUserHandshake_offsets_[5] = {
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(BitToUserHandshake, rpc_version_),
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(BitToUserHandshake, status_),
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(BitToUserHandshake, errorid_),
     GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(BitToUserHandshake, errormessage_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(BitToUserHandshake, server_infos_),
   };
   BitToUserHandshake_reflection_ =
     new ::google::protobuf::internal::GeneratedMessageReflection(
@@ -155,9 +261,363 @@ void protobuf_AssignDesc_User_2eproto() {
       ::google::protobuf::DescriptorPool::generated_pool(),
       ::google::protobuf::MessageFactory::generated_factory(),
       sizeof(BitToUserHandshake));
+  LikeFilter_descriptor_ = file->message_type(8);
+  static const int LikeFilter_offsets_[2] = {
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(LikeFilter, pattern_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(LikeFilter, escape_),
+  };
+  LikeFilter_reflection_ =
+    new ::google::protobuf::internal::GeneratedMessageReflection(
+      LikeFilter_descriptor_,
+      LikeFilter::default_instance_,
+      LikeFilter_offsets_,
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(LikeFilter, _has_bits_[0]),
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(LikeFilter, _unknown_fields_),
+      -1,
+      ::google::protobuf::DescriptorPool::generated_pool(),
+      ::google::protobuf::MessageFactory::generated_factory(),
+      sizeof(LikeFilter));
+  GetCatalogsReq_descriptor_ = file->message_type(9);
+  static const int GetCatalogsReq_offsets_[1] = {
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetCatalogsReq, catalog_name_filter_),
+  };
+  GetCatalogsReq_reflection_ =
+    new ::google::protobuf::internal::GeneratedMessageReflection(
+      GetCatalogsReq_descriptor_,
+      GetCatalogsReq::default_instance_,
+      GetCatalogsReq_offsets_,
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetCatalogsReq, _has_bits_[0]),
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetCatalogsReq, _unknown_fields_),
+      -1,
+      ::google::protobuf::DescriptorPool::generated_pool(),
+      ::google::protobuf::MessageFactory::generated_factory(),
+      sizeof(GetCatalogsReq));
+  CatalogMetadata_descriptor_ = file->message_type(10);
+  static const int CatalogMetadata_offsets_[3] = {
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(CatalogMetadata, catalog_name_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(CatalogMetadata, description_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(CatalogMetadata, connect_),
+  };
+  CatalogMetadata_reflection_ =
+    new ::google::protobuf::internal::GeneratedMessageReflection(
+      CatalogMetadata_descriptor_,
+      CatalogMetadata::default_instance_,
+      CatalogMetadata_offsets_,
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(CatalogMetadata, _has_bits_[0]),
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(CatalogMetadata, _unknown_fields_),
+      -1,
+      ::google::protobuf::DescriptorPool::generated_pool(),
+      ::google::protobuf::MessageFactory::generated_factory(),
+      sizeof(CatalogMetadata));
+  GetCatalogsResp_descriptor_ = file->message_type(11);
+  static const int GetCatalogsResp_offsets_[3] = {
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetCatalogsResp, status_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetCatalogsResp, catalogs_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetCatalogsResp, error_),
+  };
+  GetCatalogsResp_reflection_ =
+    new ::google::protobuf::internal::GeneratedMessageReflection(
+      GetCatalogsResp_descriptor_,
+      GetCatalogsResp::default_instance_,
+      GetCatalogsResp_offsets_,
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetCatalogsResp, _has_bits_[0]),
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetCatalogsResp, _unknown_fields_),
+      -1,
+      ::google::protobuf::DescriptorPool::generated_pool(),
+      ::google::protobuf::MessageFactory::generated_factory(),
+      sizeof(GetCatalogsResp));
+  GetSchemasReq_descriptor_ = file->message_type(12);
+  static const int GetSchemasReq_offsets_[2] = {
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetSchemasReq, catalog_name_filter_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetSchemasReq, schema_name_filter_),
+  };
+  GetSchemasReq_reflection_ =
+    new ::google::protobuf::internal::GeneratedMessageReflection(
+      GetSchemasReq_descriptor_,
+      GetSchemasReq::default_instance_,
+      GetSchemasReq_offsets_,
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetSchemasReq, _has_bits_[0]),
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetSchemasReq, _unknown_fields_),
+      -1,
+      ::google::protobuf::DescriptorPool::generated_pool(),
+      ::google::protobuf::MessageFactory::generated_factory(),
+      sizeof(GetSchemasReq));
+  SchemaMetadata_descriptor_ = file->message_type(13);
+  static const int SchemaMetadata_offsets_[5] = {
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(SchemaMetadata, catalog_name_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(SchemaMetadata, schema_name_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(SchemaMetadata, owner_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(SchemaMetadata, type_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(SchemaMetadata, mutable__),
+  };
+  SchemaMetadata_reflection_ =
+    new ::google::protobuf::internal::GeneratedMessageReflection(
+      SchemaMetadata_descriptor_,
+      SchemaMetadata::default_instance_,
+      SchemaMetadata_offsets_,
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(SchemaMetadata, _has_bits_[0]),
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(SchemaMetadata, _unknown_fields_),
+      -1,
+      ::google::protobuf::DescriptorPool::generated_pool(),
+      ::google::protobuf::MessageFactory::generated_factory(),
+      sizeof(SchemaMetadata));
+  GetSchemasResp_descriptor_ = file->message_type(14);
+  static const int GetSchemasResp_offsets_[3] = {
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetSchemasResp, status_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetSchemasResp, schemas_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetSchemasResp, error_),
+  };
+  GetSchemasResp_reflection_ =
+    new ::google::protobuf::internal::GeneratedMessageReflection(
+      GetSchemasResp_descriptor_,
+      GetSchemasResp::default_instance_,
+      GetSchemasResp_offsets_,
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetSchemasResp, _has_bits_[0]),
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetSchemasResp, _unknown_fields_),
+      -1,
+      ::google::protobuf::DescriptorPool::generated_pool(),
+      ::google::protobuf::MessageFactory::generated_factory(),
+      sizeof(GetSchemasResp));
+  GetTablesReq_descriptor_ = file->message_type(15);
+  static const int GetTablesReq_offsets_[4] = {
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetTablesReq, catalog_name_filter_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetTablesReq, schema_name_filter_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetTablesReq, table_name_filter_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetTablesReq, table_type_filter_),
+  };
+  GetTablesReq_reflection_ =
+    new ::google::protobuf::internal::GeneratedMessageReflection(
+      GetTablesReq_descriptor_,
+      GetTablesReq::default_instance_,
+      GetTablesReq_offsets_,
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetTablesReq, _has_bits_[0]),
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetTablesReq, _unknown_fields_),
+      -1,
+      ::google::protobuf::DescriptorPool::generated_pool(),
+      ::google::protobuf::MessageFactory::generated_factory(),
+      sizeof(GetTablesReq));
+  TableMetadata_descriptor_ = file->message_type(16);
+  static const int TableMetadata_offsets_[4] = {
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(TableMetadata, catalog_name_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(TableMetadata, schema_name_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(TableMetadata, table_name_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(TableMetadata, type_),
+  };
+  TableMetadata_reflection_ =
+    new ::google::protobuf::internal::GeneratedMessageReflection(
+      TableMetadata_descriptor_,
+      TableMetadata::default_instance_,
+      TableMetadata_offsets_,
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(TableMetadata, _has_bits_[0]),
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(TableMetadata, _unknown_fields_),
+      -1,
+      ::google::protobuf::DescriptorPool::generated_pool(),
+      ::google::protobuf::MessageFactory::generated_factory(),
+      sizeof(TableMetadata));
+  GetTablesResp_descriptor_ = file->message_type(17);
+  static const int GetTablesResp_offsets_[3] = {
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetTablesResp, status_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetTablesResp, tables_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetTablesResp, error_),
+  };
+  GetTablesResp_reflection_ =
+    new ::google::protobuf::internal::GeneratedMessageReflection(
+      GetTablesResp_descriptor_,
+      GetTablesResp::default_instance_,
+      GetTablesResp_offsets_,
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetTablesResp, _has_bits_[0]),
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetTablesResp, _unknown_fields_),
+      -1,
+      ::google::protobuf::DescriptorPool::generated_pool(),
+      ::google::protobuf::MessageFactory::generated_factory(),
+      sizeof(GetTablesResp));
+  GetColumnsReq_descriptor_ = file->message_type(18);
+  static const int GetColumnsReq_offsets_[4] = {
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetColumnsReq, catalog_name_filter_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetColumnsReq, schema_name_filter_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetColumnsReq, table_name_filter_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetColumnsReq, column_name_filter_),
+  };
+  GetColumnsReq_reflection_ =
+    new ::google::protobuf::internal::GeneratedMessageReflection(
+      GetColumnsReq_descriptor_,
+      GetColumnsReq::default_instance_,
+      GetColumnsReq_offsets_,
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetColumnsReq, _has_bits_[0]),
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetColumnsReq, _unknown_fields_),
+      -1,
+      ::google::protobuf::DescriptorPool::generated_pool(),
+      ::google::protobuf::MessageFactory::generated_factory(),
+      sizeof(GetColumnsReq));
+  ColumnMetadata_descriptor_ = file->message_type(19);
+  static const int ColumnMetadata_offsets_[17] = {
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ColumnMetadata, catalog_name_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ColumnMetadata, schema_name_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ColumnMetadata, table_name_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ColumnMetadata, column_name_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ColumnMetadata, ordinal_position_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ColumnMetadata, default_value_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ColumnMetadata, is_nullable_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ColumnMetadata, data_type_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ColumnMetadata, char_max_length_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ColumnMetadata, char_octet_length_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ColumnMetadata, numeric_precision_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ColumnMetadata, numeric_precision_radix_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ColumnMetadata, numeric_scale_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ColumnMetadata, date_time_precision_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ColumnMetadata, interval_type_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ColumnMetadata, interval_precision_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ColumnMetadata, column_size_),
+  };
+  ColumnMetadata_reflection_ =
+    new ::google::protobuf::internal::GeneratedMessageReflection(
+      ColumnMetadata_descriptor_,
+      ColumnMetadata::default_instance_,
+      ColumnMetadata_offsets_,
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ColumnMetadata, _has_bits_[0]),
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ColumnMetadata, _unknown_fields_),
+      -1,
+      ::google::protobuf::DescriptorPool::generated_pool(),
+      ::google::protobuf::MessageFactory::generated_factory(),
+      sizeof(ColumnMetadata));
+  GetColumnsResp_descriptor_ = file->message_type(20);
+  static const int GetColumnsResp_offsets_[3] = {
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetColumnsResp, status_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetColumnsResp, columns_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetColumnsResp, error_),
+  };
+  GetColumnsResp_reflection_ =
+    new ::google::protobuf::internal::GeneratedMessageReflection(
+      GetColumnsResp_descriptor_,
+      GetColumnsResp::default_instance_,
+      GetColumnsResp_offsets_,
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetColumnsResp, _has_bits_[0]),
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(GetColumnsResp, _unknown_fields_),
+      -1,
+      ::google::protobuf::DescriptorPool::generated_pool(),
+      ::google::protobuf::MessageFactory::generated_factory(),
+      sizeof(GetColumnsResp));
+  CreatePreparedStatementReq_descriptor_ = file->message_type(21);
+  static const int CreatePreparedStatementReq_offsets_[1] = {
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(CreatePreparedStatementReq, sql_query_),
+  };
+  CreatePreparedStatementReq_reflection_ =
+    new ::google::protobuf::internal::GeneratedMessageReflection(
+      CreatePreparedStatementReq_descriptor_,
+      CreatePreparedStatementReq::default_instance_,
+      CreatePreparedStatementReq_offsets_,
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(CreatePreparedStatementReq, _has_bits_[0]),
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(CreatePreparedStatementReq, _unknown_fields_),
+      -1,
+      ::google::protobuf::DescriptorPool::generated_pool(),
+      ::google::protobuf::MessageFactory::generated_factory(),
+      sizeof(CreatePreparedStatementReq));
+  ResultColumnMetadata_descriptor_ = file->message_type(22);
+  static const int ResultColumnMetadata_offsets_[19] = {
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ResultColumnMetadata, catalog_name_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ResultColumnMetadata, schema_name_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ResultColumnMetadata, table_name_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ResultColumnMetadata, column_name_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ResultColumnMetadata, label_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ResultColumnMetadata, data_type_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ResultColumnMetadata, is_nullable_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ResultColumnMetadata, precision_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ResultColumnMetadata, scale_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ResultColumnMetadata, signed__),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ResultColumnMetadata, display_size_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ResultColumnMetadata, is_aliased_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ResultColumnMetadata, searchability_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ResultColumnMetadata, updatability_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ResultColumnMetadata, auto_increment_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ResultColumnMetadata, case_sensitivity_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ResultColumnMetadata, sortable_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ResultColumnMetadata, class_name_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ResultColumnMetadata, is_currency_),
+  };
+  ResultColumnMetadata_reflection_ =
+    new ::google::protobuf::internal::GeneratedMessageReflection(
+      ResultColumnMetadata_descriptor_,
+      ResultColumnMetadata::default_instance_,
+      ResultColumnMetadata_offsets_,
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ResultColumnMetadata, _has_bits_[0]),
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(ResultColumnMetadata, _unknown_fields_),
+      -1,
+      ::google::protobuf::DescriptorPool::generated_pool(),
+      ::google::protobuf::MessageFactory::generated_factory(),
+      sizeof(ResultColumnMetadata));
+  PreparedStatementHandle_descriptor_ = file->message_type(23);
+  static const int PreparedStatementHandle_offsets_[1] = {
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(PreparedStatementHandle, server_info_),
+  };
+  PreparedStatementHandle_reflection_ =
+    new ::google::protobuf::internal::GeneratedMessageReflection(
+      PreparedStatementHandle_descriptor_,
+      PreparedStatementHandle::default_instance_,
+      PreparedStatementHandle_offsets_,
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(PreparedStatementHandle, _has_bits_[0]),
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(PreparedStatementHandle, _unknown_fields_),
+      -1,
+      ::google::protobuf::DescriptorPool::generated_pool(),
+      ::google::protobuf::MessageFactory::generated_factory(),
+      sizeof(PreparedStatementHandle));
+  PreparedStatement_descriptor_ = file->message_type(24);
+  static const int PreparedStatement_offsets_[2] = {
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(PreparedStatement, columns_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(PreparedStatement, server_handle_),
+  };
+  PreparedStatement_reflection_ =
+    new ::google::protobuf::internal::GeneratedMessageReflection(
+      PreparedStatement_descriptor_,
+      PreparedStatement::default_instance_,
+      PreparedStatement_offsets_,
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(PreparedStatement, _has_bits_[0]),
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(PreparedStatement, _unknown_fields_),
+      -1,
+      ::google::protobuf::DescriptorPool::generated_pool(),
+      ::google::protobuf::MessageFactory::generated_factory(),
+      sizeof(PreparedStatement));
+  CreatePreparedStatementResp_descriptor_ = file->message_type(25);
+  static const int CreatePreparedStatementResp_offsets_[3] = {
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(CreatePreparedStatementResp, status_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(CreatePreparedStatementResp, prepared_statement_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(CreatePreparedStatementResp, error_),
+  };
+  CreatePreparedStatementResp_reflection_ =
+    new ::google::protobuf::internal::GeneratedMessageReflection(
+      CreatePreparedStatementResp_descriptor_,
+      CreatePreparedStatementResp::default_instance_,
+      CreatePreparedStatementResp_offsets_,
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(CreatePreparedStatementResp, _has_bits_[0]),
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(CreatePreparedStatementResp, _unknown_fields_),
+      -1,
+      ::google::protobuf::DescriptorPool::generated_pool(),
+      ::google::protobuf::MessageFactory::generated_factory(),
+      sizeof(CreatePreparedStatementResp));
+  RunQuery_descriptor_ = file->message_type(26);
+  static const int RunQuery_offsets_[5] = {
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(RunQuery, results_mode_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(RunQuery, type_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(RunQuery, plan_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(RunQuery, fragments_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(RunQuery, prepared_statement_handle_),
+  };
+  RunQuery_reflection_ =
+    new ::google::protobuf::internal::GeneratedMessageReflection(
+      RunQuery_descriptor_,
+      RunQuery::default_instance_,
+      RunQuery_offsets_,
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(RunQuery, _has_bits_[0]),
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(RunQuery, _unknown_fields_),
+      -1,
+      ::google::protobuf::DescriptorPool::generated_pool(),
+      ::google::protobuf::MessageFactory::generated_factory(),
+      sizeof(RunQuery));
   RpcType_descriptor_ = file->enum_type(0);
   QueryResultsMode_descriptor_ = file->enum_type(1);
   HandshakeStatus_descriptor_ = file->enum_type(2);
+  RequestStatus_descriptor_ = file->enum_type(3);
+  ColumnSearchability_descriptor_ = file->enum_type(4);
+  ColumnUpdatability_descriptor_ = file->enum_type(5);
 }
 
 namespace {
@@ -175,13 +635,55 @@ void protobuf_RegisterTypes(const ::std::string&) {
   ::google::protobuf::MessageFactory::InternalRegisterGeneratedMessage(
     UserProperties_descriptor_, &UserProperties::default_instance());
   ::google::protobuf::MessageFactory::InternalRegisterGeneratedMessage(
+    RpcEndpointInfos_descriptor_, &RpcEndpointInfos::default_instance());
+  ::google::protobuf::MessageFactory::InternalRegisterGeneratedMessage(
     UserToBitHandshake_descriptor_, &UserToBitHandshake::default_instance());
   ::google::protobuf::MessageFactory::InternalRegisterGeneratedMessage(
     RequestResults_descriptor_, &RequestResults::default_instance());
   ::google::protobuf::MessageFactory::InternalRegisterGeneratedMessage(
-    RunQuery_descriptor_, &RunQuery::default_instance());
+    GetQueryPlanFragments_descriptor_, &GetQueryPlanFragments::default_instance());
+  ::google::protobuf::MessageFactory::InternalRegisterGeneratedMessage(
+    QueryPlanFragments_descriptor_, &QueryPlanFragments::default_instance());
   ::google::protobuf::MessageFactory::InternalRegisterGeneratedMessage(
     BitToUserHandshake_descriptor_, &BitToUserHandshake::default_instance());
+  ::google::protobuf::MessageFactory::InternalRegisterGeneratedMessage(
+    LikeFilter_descriptor_, &LikeFilter::default_instance());
+  ::google::protobuf::MessageFactory::InternalRegisterGeneratedMessage(
+    GetCatalogsReq_descriptor_, &GetCatalogsReq::default_instance());
+  ::google::protobuf::MessageFactory::InternalRegisterGeneratedMessage(
+    CatalogMetadata_descriptor_, &CatalogMetadata::default_instance());
+  ::google::protobuf::MessageFactory::InternalRegisterGeneratedMessage(
+    GetCatalogsResp_descriptor_, &GetCatalogsResp::default_instance());
+  ::google::protobuf::MessageFactory::InternalRegisterGeneratedMessage(
+    GetSchemasReq_descriptor_, &GetSchemasReq::default_instance());
+  ::google::protobuf::MessageFactory::InternalRegisterGeneratedMessage(
+    SchemaMetadata_descriptor_, &SchemaMetadata::default_instance());
+  ::google::protobuf::MessageFactory::InternalRegisterGeneratedMessage(
+    GetSchemasResp_descriptor_, &GetSchemasResp::default_instance());
+  ::google::protobuf::MessageFactory::InternalRegisterGeneratedMessage(
+    GetTablesReq_descriptor_, &GetTablesReq::default_instance());
+  ::google::protobuf::MessageFactory::InternalRegisterGeneratedMessage(
+    TableMetadata_descriptor_, &TableMetadata::default_instance());
+  ::google::protobuf::MessageFactory::InternalRegisterGeneratedMessage(
+    GetTablesResp_descriptor_, &GetTablesResp::default_instance());
+  ::google::protobuf::MessageFactory::InternalRegisterGeneratedMessage(
+    GetColumnsReq_descriptor_, &GetColumnsReq::default_instance());
+  ::google::protobuf::MessageFactory::InternalRegisterGeneratedMessage(
+    ColumnMetadata_descriptor_, &ColumnMetadata::default_instance());
+  ::google::protobuf::MessageFactory::InternalRegisterGeneratedMessage(
+    GetColumnsResp_descriptor_, &GetColumnsResp::default_instance());
+  ::google::protobuf::MessageFactory::InternalRegisterGeneratedMessage(
+    CreatePreparedStatementReq_descriptor_, &CreatePreparedStatementReq::default_instance());
+  ::google::protobuf::MessageFactory::InternalRegisterGeneratedMessage(
+    ResultColumnMetadata_descriptor_, &ResultColumnMetadata::default_instance());
+  ::google::protobuf::MessageFactory::InternalRegisterGeneratedMessage(
+    PreparedStatementHandle_descriptor_, &PreparedStatementHandle::default_instance());
+  ::google::protobuf::MessageFactory::InternalRegisterGeneratedMessage(
+    PreparedStatement_descriptor_, &PreparedStatement::default_instance());
+  ::google::protobuf::MessageFactory::InternalRegisterGeneratedMessage(
+    CreatePreparedStatementResp_descriptor_, &CreatePreparedStatementResp::default_instance());
+  ::google::protobuf::MessageFactory::InternalRegisterGeneratedMessage(
+    RunQuery_descriptor_, &RunQuery::default_instance());
 }
 
 }  // namespace
@@ -191,14 +693,56 @@ void protobuf_ShutdownFile_User_2eproto() {
   delete Property_reflection_;
   delete UserProperties::default_instance_;
   delete UserProperties_reflection_;
+  delete RpcEndpointInfos::default_instance_;
+  delete RpcEndpointInfos_reflection_;
   delete UserToBitHandshake::default_instance_;
   delete UserToBitHandshake_reflection_;
   delete RequestResults::default_instance_;
   delete RequestResults_reflection_;
-  delete RunQuery::default_instance_;
-  delete RunQuery_reflection_;
+  delete GetQueryPlanFragments::default_instance_;
+  delete GetQueryPlanFragments_reflection_;
+  delete QueryPlanFragments::default_instance_;
+  delete QueryPlanFragments_reflection_;
   delete BitToUserHandshake::default_instance_;
   delete BitToUserHandshake_reflection_;
+  delete LikeFilter::default_instance_;
+  delete LikeFilter_reflection_;
+  delete GetCatalogsReq::default_instance_;
+  delete GetCatalogsReq_reflection_;
+  delete CatalogMetadata::default_instance_;
+  delete CatalogMetadata_reflection_;
+  delete GetCatalogsResp::default_instance_;
+  delete GetCatalogsResp_reflection_;
+  delete GetSchemasReq::default_instance_;
+  delete GetSchemasReq_reflection_;
+  delete SchemaMetadata::default_instance_;
+  delete SchemaMetadata_reflection_;
+  delete GetSchemasResp::default_instance_;
+  delete GetSchemasResp_reflection_;
+  delete GetTablesReq::default_instance_;
+  delete GetTablesReq_reflection_;
+  delete TableMetadata::default_instance_;
+  delete TableMetadata_reflection_;
+  delete GetTablesResp::default_instance_;
+  delete GetTablesResp_reflection_;
+  delete GetColumnsReq::default_instance_;
+  delete GetColumnsReq_reflection_;
+  delete ColumnMetadata::default_instance_;
+  delete ColumnMetadata_reflection_;
+  delete GetColumnsResp::default_instance_;
+  delete GetColumnsResp_reflection_;
+  delete CreatePreparedStatementReq::default_instance_;
+  delete CreatePreparedStatementReq_reflection_;
+  delete ResultColumnMetadata::default_instance_;
+  delete ResultColumnMetadata_reflection_;
+  delete PreparedStatementHandle::default_instance_;
+  delete PreparedStatementHandle_reflection_;
+  delete PreparedStatement::default_instance_;
+  delete PreparedStatement_reflection_;
+  delete CreatePreparedStatementResp::default_instance_;
+  delete CreatePreparedStatementResp_reflection_;
+  delete RunQuery::default_instance_;
+  delete RunQuery_reflection_;
 }
 
 void protobuf_AddDesc_User_2eproto() {
@@ -209,51 +753,197 @@ void protobuf_AddDesc_User_2eproto() {
 
   ::exec::protobuf_AddDesc_SchemaDef_2eproto();
   ::exec::shared::protobuf_AddDesc_UserBitShared_2eproto();
+  ::exec::bit::data::protobuf_AddDesc_BitData_2eproto();
+  ::exec::bit::control::protobuf_AddDesc_BitControl_2eproto();
+  ::exec::bit::protobuf_AddDesc_ExecutionProtos_2eproto();
   ::google::protobuf::DescriptorPool::InternalAddGeneratedFile(
     "\n\nUser.proto\022\texec.user\032\017SchemaDef.proto"
-    "\032\023UserBitShared.proto\"&\n\010Property\022\013\n\003key"
-    "\030\001 \002(\t\022\r\n\005value\030\002 \002(\t\"9\n\016UserProperties\022"
-    "\'\n\nproperties\030\001 \003(\0132\023.exec.user.Property"
-    "\"\234\002\n\022UserToBitHandshake\022.\n\007channel\030\001 \001(\016"
-    "2\027.exec.shared.RpcChannel:\004USER\022\031\n\021suppo"
-    "rt_listening\030\002 \001(\010\022\023\n\013rpc_version\030\003 \001(\005\022"
-    "1\n\013credentials\030\004 \001(\0132\034.exec.shared.UserC"
-    "redentials\022-\n\nproperties\030\005 \001(\0132\031.exec.us"
-    "er.UserProperties\022$\n\025support_complex_typ"
-    "es\030\006 \001(\010:\005false\022\036\n\017support_timeout\030\007 \001(\010"
-    ":\005false\"S\n\016RequestResults\022&\n\010query_id\030\001 "
-    "\001(\0132\024.exec.shared.QueryId\022\031\n\021maximum_res"
-    "ponses\030\002 \001(\005\"q\n\010RunQuery\0221\n\014results_mode"
-    "\030\001 \001(\0162\033.exec.user.QueryResultsMode\022$\n\004t"
-    "ype\030\002 \001(\0162\026.exec.shared.QueryType\022\014\n\004pla"
-    "n\030\003 \001(\t\"|\n\022BitToUserHandshake\022\023\n\013rpc_ver"
-    "sion\030\002 \001(\005\022*\n\006status\030\003 \001(\0162\032.exec.user.H"
-    "andshakeStatus\022\017\n\007errorId\030\004 \001(\t\022\024\n\014error"
-    "Message\030\005 \001(\t*\341\001\n\007RpcType\022\r\n\tHANDSHAKE\020\000"
-    "\022\007\n\003ACK\020\001\022\013\n\007GOODBYE\020\002\022\r\n\tRUN_QUERY\020\003\022\020\n"
-    "\014CANCEL_QUERY\020\004\022\023\n\017REQUEST_RESULTS\020\005\022\027\n\023"
-    "RESUME_PAUSED_QUERY\020\013\022\016\n\nQUERY_DATA\020\006\022\020\n"
-    "\014QUERY_HANDLE\020\007\022\026\n\022REQ_META_FUNCTIONS\020\010\022"
-    "\026\n\022RESP_FUNCTION_LIST\020\t\022\020\n\014QUERY_RESULT\020"
-    "\n*#\n\020QueryResultsMode\022\017\n\013STREAM_FULL\020\001*^"
-    "\n\017HandshakeStatus\022\013\n\007SUCCESS\020\001\022\030\n\024RPC_VE"
-    "RSION_MISMATCH\020\002\022\017\n\013AUTH_FAILED\020\003\022\023\n\017UNK"
-    "NOWN_FAILURE\020\004B+\n\033org.apache.drill.exec."
-    "protoB\nUserProtosH\001", 1179);
+    "\032\023UserBitShared.proto\032\rBitData.proto\032\020Bi"
+    "tControl.proto\032\025ExecutionProtos.proto\"&\n"
+    "\010Property\022\013\n\003key\030\001 \002(\t\022\r\n\005value\030\002 \002(\t\"9\n"
+    "\016UserProperties\022\'\n\nproperties\030\001 \003(\0132\023.ex"
+    "ec.user.Property\"\210\001\n\020RpcEndpointInfos\022\014\n"
+    "\004name\030\001 \001(\t\022\017\n\007version\030\002 \001(\t\022\024\n\014majorVer"
+    "sion\030\003 \001(\r\022\024\n\014minorVersion\030\004 \001(\r\022\024\n\014patc"
+    "hVersion\030\005 \001(\r\022\023\n\013application\030\006 \001(\t\"\317\002\n\022"
+    "UserToBitHandshake\022.\n\007channel\030\001 \001(\0162\027.ex"
+    "ec.shared.RpcChannel:\004USER\022\031\n\021support_li"
+    "stening\030\002 \001(\010\022\023\n\013rpc_version\030\003 \001(\005\0221\n\013cr"
+    "edentials\030\004 \001(\0132\034.exec.shared.UserCreden"
+    "tials\022-\n\nproperties\030\005 \001(\0132\031.exec.user.Us"
+    "erProperties\022$\n\025support_complex_types\030\006 "
+    "\001(\010:\005false\022\036\n\017support_timeout\030\007 \001(\010:\005fal"
+    "se\0221\n\014client_infos\030\010 \001(\0132\033.exec.user.Rpc"
+    "EndpointInfos\"S\n\016RequestResults\022&\n\010query"
+    "_id\030\001 \001(\0132\024.exec.shared.QueryId\022\031\n\021maxim"
+    "um_responses\030\002 \001(\005\"g\n\025GetQueryPlanFragme"
+    "nts\022\r\n\005query\030\001 \002(\t\022$\n\004type\030\002 \001(\0162\026.exec."
+    "shared.QueryType\022\031\n\nsplit_plan\030\003 \001(\010:\005fa"
+    "lse\"\316\001\n\022QueryPlanFragments\0223\n\006status\030\001 \002"
+    "(\0162#.exec.shared.QueryResult.QueryState\022"
+    "&\n\010query_id\030\002 \001(\0132\024.exec.shared.QueryId\022"
+    "1\n\tfragments\030\003 \003(\0132\036.exec.bit.control.Pl"
+    "anFragment\022(\n\005error\030\004 \001(\0132\031.exec.shared."
+    "DrillPBError\"\257\001\n\022BitToUserHandshake\022\023\n\013r"
+    "pc_version\030\002 \001(\005\022*\n\006status\030\003 \001(\0162\032.exec."
+    "user.HandshakeStatus\022\017\n\007errorId\030\004 \001(\t\022\024\n"
+    "\014errorMessage\030\005 \001(\t\0221\n\014server_infos\030\006 \001("
+    "\0132\033.exec.user.RpcEndpointInfos\"-\n\nLikeFi"
+    "lter\022\017\n\007pattern\030\001 \001(\t\022\016\n\006escape\030\002 \001(\t\"D\n"
+    "\016GetCatalogsReq\0222\n\023catalog_name_filter\030\001"
+    " \001(\0132\025.exec.user.LikeFilter\"M\n\017CatalogMe"
+    "tadata\022\024\n\014catalog_name\030\001 \001(\t\022\023\n\013descript"
+    "ion\030\002 \001(\t\022\017\n\007connect\030\003 \001(\t\"\223\001\n\017GetCatalo"
+    "gsResp\022(\n\006status\030\001 \001(\0162\030.exec.user.Reque"
+    "stStatus\022,\n\010catalogs\030\002 \003(\0132\032.exec.user.C"
+    "atalogMetadata\022(\n\005error\030\003 \001(\0132\031.exec.sha"
+    "red.DrillPBError\"v\n\rGetSchemasReq\0222\n\023cat"
+    "alog_name_filter\030\001 \001(\0132\025.exec.user.LikeF"
+    "ilter\0221\n\022schema_name_filter\030\002 \001(\0132\025.exec"
+    ".user.LikeFilter\"i\n\016SchemaMetadata\022\024\n\014ca"
+    "talog_name\030\001 \001(\t\022\023\n\013schema_name\030\002 \001(\t\022\r\n"
+    "\005owner\030\003 \001(\t\022\014\n\004type\030\004 \001(\t\022\017\n\007mutable\030\005 "
+    "\001(\t\"\220\001\n\016GetSchemasResp\022(\n\006status\030\001 \001(\0162\030"
+    ".exec.user.RequestStatus\022*\n\007schemas\030\002 \003("
+    "\0132\031.exec.user.SchemaMetadata\022(\n\005error\030\003 "
+    "\001(\0132\031.exec.shared.DrillPBError\"\302\001\n\014GetTa"
+    "blesReq\0222\n\023catalog_name_filter\030\001 \001(\0132\025.e"
+    "xec.user.LikeFilter\0221\n\022schema_name_filte"
+    "r\030\002 \001(\0132\025.exec.user.LikeFilter\0220\n\021table_"
+    "name_filter\030\003 \001(\0132\025.exec.user.LikeFilter"
+    "\022\031\n\021table_type_filter\030\004 \003(\t\"\\\n\rTableMeta"
+    "data\022\024\n\014catalog_name\030\001 \001(\t\022\023\n\013schema_nam"
+    "e\030\002 \001(\t\022\022\n\ntable_name\030\003 \001(\t\022\014\n\004type\030\004 \001("
+    "\t\"\215\001\n\rGetTablesResp\022(\n\006status\030\001 \001(\0162\030.ex"
+    "ec.user.RequestStatus\022(\n\006tables\030\002 \003(\0132\030."
+    "exec.user.TableMetadata\022(\n\005error\030\003 \001(\0132\031"
+    ".exec.shared.DrillPBError\"\333\001\n\rGetColumns"
+    "Req\0222\n\023catalog_name_filter\030\001 \001(\0132\025.exec."
+    "user.LikeFilter\0221\n\022schema_name_filter\030\002 "
+    "\001(\0132\025.exec.user.LikeFilter\0220\n\021table_name"
+    "_filter\030\003 \001(\0132\025.exec.user.LikeFilter\0221\n\022"
+    "column_name_filter\030\004 \001(\0132\025.exec.user.Lik"
+    "eFilter\"\251\003\n\016ColumnMetadata\022\024\n\014catalog_na"
+    "me\030\001 \001(\t\022\023\n\013schema_name\030\002 \001(\t\022\022\n\ntable_n"
+    "ame\030\003 \001(\t\022\023\n\013column_name\030\004 \001(\t\022\030\n\020ordina"
+    "l_position\030\005 \001(\005\022\025\n\rdefault_value\030\006 \001(\t\022"
+    "\023\n\013is_nullable\030\007 \001(\010\022\021\n\tdata_type\030\010 \001(\t\022"
+    "\027\n\017char_max_length\030\t \001(\005\022\031\n\021char_octet_l"
+    "ength\030\n \001(\005\022\031\n\021numeric_precision\030\013 \001(\005\022\037"
+    "\n\027numeric_precision_radix\030\014 \001(\005\022\025\n\rnumer"
+    "ic_scale\030\r \001(\005\022\033\n\023date_time_precision\030\016 "
+    "\001(\005\022\025\n\rinterval_type\030\017 \001(\t\022\032\n\022interval_p"
+    "recision\030\020 \001(\005\022\023\n\013column_size\030\021 \001(\005\"\220\001\n\016"
+    "GetColumnsResp\022(\n\006status\030\001 \001(\0162\030.exec.us"
+    "er.RequestStatus\022*\n\007columns\030\002 \003(\0132\031.exec"
+    ".user.ColumnMetadata\022(\n\005error\030\003 \001(\0132\031.ex"
+    "ec.shared.DrillPBError\"/\n\032CreatePrepared"
+    "StatementReq\022\021\n\tsql_query\030\001 \001(\t\"\326\003\n\024Resu"
+    "ltColumnMetadata\022\024\n\014catalog_name\030\001 \001(\t\022\023"
+    "\n\013schema_name\030\002 \001(\t\022\022\n\ntable_name\030\003 \001(\t\022"
+    "\023\n\013column_name\030\004 \001(\t\022\r\n\005label\030\005 \001(\t\022\021\n\td"
+    "ata_type\030\006 \001(\t\022\023\n\013is_nullable\030\007 \001(\010\022\021\n\tp"
+    "recision\030\010 \001(\005\022\r\n\005scale\030\t \001(\005\022\016\n\006signed\030"
+    "\n \001(\010\022\024\n\014display_size\030\013 \001(\005\022\022\n\nis_aliase"
+    "d\030\014 \001(\010\0225\n\rsearchability\030\r \001(\0162\036.exec.us"
+    "er.ColumnSearchability\0223\n\014updatability\030\016"
+    " \001(\0162\035.exec.user.ColumnUpdatability\022\026\n\016a"
+    "uto_increment\030\017 \001(\010\022\030\n\020case_sensitivity\030"
+    "\020 \001(\010\022\020\n\010sortable\030\021 \001(\010\022\022\n\nclass_name\030\022 "
+    "\001(\t\022\023\n\013is_currency\030\024 \001(\010\".\n\027PreparedStat"
+    "ementHandle\022\023\n\013server_info\030\001 \001(\014\"\200\001\n\021Pre"
+    "paredStatement\0220\n\007columns\030\001 \003(\0132\037.exec.u"
+    "ser.ResultColumnMetadata\0229\n\rserver_handl"
+    "e\030\002 \001(\0132\".exec.user.PreparedStatementHan"
+    "dle\"\253\001\n\033CreatePreparedStatementResp\022(\n\006s"
+    "tatus\030\001 \001(\0162\030.exec.user.RequestStatus\0228\n"
+    "\022prepared_statement\030\002 \001(\0132\034.exec.user.Pr"
+    "eparedStatement\022(\n\005error\030\003 \001(\0132\031.exec.sh"
+    "ared.DrillPBError\"\353\001\n\010RunQuery\0221\n\014result"
+    "s_mode\030\001 \001(\0162\033.exec.user.QueryResultsMod"
+    "e\022$\n\004type\030\002 \001(\0162\026.exec.shared.QueryType\022"
+    "\014\n\004plan\030\003 \001(\t\0221\n\tfragments\030\004 \003(\0132\036.exec."
+    "bit.control.PlanFragment\022E\n\031prepared_sta"
+    "tement_handle\030\005 \001(\0132\".exec.user.Prepared"
+    "StatementHandle*\310\003\n\007RpcType\022\r\n\tHANDSHAKE"
+    "\020\000\022\007\n\003ACK\020\001\022\013\n\007GOODBYE\020\002\022\r\n\tRUN_QUERY\020\003\022"
+    "\020\n\014CANCEL_QUERY\020\004\022\023\n\017REQUEST_RESULTS\020\005\022\027"
+    "\n\023RESUME_PAUSED_QUERY\020\013\022\034\n\030GET_QUERY_PLA"
+    "N_FRAGMENTS\020\014\022\020\n\014GET_CATALOGS\020\016\022\017\n\013GET_S"
+    "CHEMAS\020\017\022\016\n\nGET_TABLES\020\020\022\017\n\013GET_COLUMNS\020"
+    "\021\022\035\n\031CREATE_PREPARED_STATEMENT\020\026\022\016\n\nQUER"
+    "Y_DATA\020\006\022\020\n\014QUERY_HANDLE\020\007\022\030\n\024QUERY_PLAN"
+    "_FRAGMENTS\020\r\022\014\n\010CATALOGS\020\022\022\013\n\007SCHEMAS\020\023\022"
+    "\n\n\006TABLES\020\024\022\013\n\007COLUMNS\020\025\022\026\n\022PREPARED_STA"
+    "TEMENT\020\027\022\026\n\022REQ_META_FUNCTIONS\020\010\022\026\n\022RESP"
+    "_FUNCTION_LIST\020\t\022\020\n\014QUERY_RESULT\020\n*#\n\020Qu"
+    "eryResultsMode\022\017\n\013STREAM_FULL\020\001*^\n\017Hands"
+    "hakeStatus\022\013\n\007SUCCESS\020\001\022\030\n\024RPC_VERSION_M"
+    "ISMATCH\020\002\022\017\n\013AUTH_FAILED\020\003\022\023\n\017UNKNOWN_FA"
+    "ILURE\020\004*D\n\rRequestStatus\022\022\n\016UNKNOWN_STAT"
+    "US\020\000\022\006\n\002OK\020\001\022\n\n\006FAILED\020\002\022\013\n\007TIMEOUT\020\003*Y\n"
+    "\023ColumnSearchability\022\031\n\025UNKNOWN_SEARCHAB"
+    "ILITY\020\000\022\010\n\004NONE\020\001\022\010\n\004CHAR\020\002\022\n\n\006NUMBER\020\003\022"
+    "\007\n\003ALL\020\004*K\n\022ColumnUpdatability\022\030\n\024UNKNOW"
+    "N_UPDATABILITY\020\000\022\r\n\tREAD_ONLY\020\001\022\014\n\010WRITA"
+    "BLE\020\002B+\n\033org.apache.drill.exec.protoB\nUs"
+    "erProtosH\001", 5210);
   ::google::protobuf::MessageFactory::InternalRegisterGeneratedFile(
     "User.proto", &protobuf_RegisterTypes);
   Property::default_instance_ = new Property();
   UserProperties::default_instance_ = new UserProperties();
+  RpcEndpointInfos::default_instance_ = new RpcEndpointInfos();
   UserToBitHandshake::default_instance_ = new UserToBitHandshake();
   RequestResults::default_instance_ = new RequestResults();
-  RunQuery::default_instance_ = new RunQuery();
+  GetQueryPlanFragments::default_instance_ = new GetQueryPlanFragments();
+  QueryPlanFragments::default_instance_ = new QueryPlanFragments();
   BitToUserHandshake::default_instance_ = new BitToUserHandshake();
+  LikeFilter::default_instance_ = new LikeFilter();
+  GetCatalogsReq::default_instance_ = new GetCatalogsReq();
+  CatalogMetadata::default_instance_ = new CatalogMetadata();
+  GetCatalogsResp::default_instance_ = new GetCatalogsResp();
+  GetSchemasReq::default_instance_ = new GetSchemasReq();
+  SchemaMetadata::default_instance_ = new SchemaMetadata();
+  GetSchemasResp::default_instance_ = new GetSchemasResp();
+  GetTablesReq::default_instance_ = new GetTablesReq();
+  TableMetadata::default_instance_ = new TableMetadata();
+  GetTablesResp::default_instance_ = new GetTablesResp();
+  GetColumnsReq::default_instance_ = new GetColumnsReq();
+  ColumnMetadata::default_instance_ = new ColumnMetadata();
+  GetColumnsResp::default_instance_ = new GetColumnsResp();
+  CreatePreparedStatementReq::default_instance_ = new CreatePreparedStatementReq();
+  ResultColumnMetadata::default_instance_ = new ResultColumnMetadata();
+  PreparedStatementHandle::default_instance_ = new PreparedStatementHandle();
+  PreparedStatement::default_instance_ = new PreparedStatement();
+  CreatePreparedStatementResp::default_instance_ = new CreatePreparedStatementResp();
+  RunQuery::default_instance_ = new RunQuery();
   Property::default_instance_->InitAsDefaultInstance();
   UserProperties::default_instance_->InitAsDefaultInstance();
+  RpcEndpointInfos::default_instance_->InitAsDefaultInstance();
   UserToBitHandshake::default_instance_->InitAsDefaultInstance();
   RequestResults::default_instance_->InitAsDefaultInstance();
-  RunQuery::default_instance_->InitAsDefaultInstance();
+  GetQueryPlanFragments::default_instance_->InitAsDefaultInstance();
+  QueryPlanFragments::default_instance_->InitAsDefaultInstance();
   BitToUserHandshake::default_instance_->InitAsDefaultInstance();
+  LikeFilter::default_instance_->InitAsDefaultInstance();
+  GetCatalogsReq::default_instance_->InitAsDefaultInstance();
+  CatalogMetadata::default_instance_->InitAsDefaultInstance();
+  GetCatalogsResp::default_instance_->InitAsDefaultInstance();
+  GetSchemasReq::default_instance_->InitAsDefaultInstance();
+  SchemaMetadata::default_instance_->InitAsDefaultInstance();
+  GetSchemasResp::default_instance_->InitAsDefaultInstance();
+  GetTablesReq::default_instance_->InitAsDefaultInstance();
+  TableMetadata::default_instance_->InitAsDefaultInstance();
+  GetTablesResp::default_instance_->InitAsDefaultInstance();
+  GetColumnsReq::default_instance_->InitAsDefaultInstance();
+  ColumnMetadata::default_instance_->InitAsDefaultInstance();
+  GetColumnsResp::default_instance_->InitAsDefaultInstance();
+  CreatePreparedStatementReq::default_instance_->InitAsDefaultInstance();
+  ResultColumnMetadata::default_instance_->InitAsDefaultInstance();
+  PreparedStatementHandle::default_instance_->InitAsDefaultInstance();
+  PreparedStatement::default_instance_->InitAsDefaultInstance();
+  CreatePreparedStatementResp::default_instance_->InitAsDefaultInstance();
+  RunQuery::default_instance_->InitAsDefaultInstance();
   ::google::protobuf::internal::OnShutdown(&protobuf_ShutdownFile_User_2eproto);
 }
 
@@ -281,6 +971,18 @@ bool RpcType_IsValid(int value) {
     case 9:
     case 10:
     case 11:
+    case 12:
+    case 13:
+    case 14:
+    case 15:
+    case 16:
+    case 17:
+    case 18:
+    case 19:
+    case 20:
+    case 21:
+    case 22:
+    case 23:
       return true;
     default:
       return false;
@@ -316,6 +1018,54 @@ bool HandshakeStatus_IsValid(int value) {
   }
 }
 
+const ::google::protobuf::EnumDescriptor* RequestStatus_descriptor() {
+  protobuf_AssignDescriptorsOnce();
+  return RequestStatus_descriptor_;
+}
+bool RequestStatus_IsValid(int value) {
+  switch(value) {
+    case 0:
+    case 1:
+    case 2:
+    case 3:
+      return true;
+    default:
+      return false;
+  }
+}
+
+const ::google::protobuf::EnumDescriptor* ColumnSearchability_descriptor() {
+  protobuf_AssignDescriptorsOnce();
+  return ColumnSearchability_descriptor_;
+}
+bool ColumnSearchability_IsValid(int value) {
+  switch(value) {
+    case 0:
+    case 1:
+    case 2:
+    case 3:
+    case 4:
+      return true;
+    default:
+      return false;
+  }
+}
+
+const ::google::protobuf::EnumDescriptor* ColumnUpdatability_descriptor() {
+  protobuf_AssignDescriptorsOnce();
+  return ColumnUpdatability_descriptor_;
+}
+bool ColumnUpdatability_IsValid(int value) {
+  switch(value) {
+    case 0:
+    case 1:
+    case 2:
+      return true;
+    default:
+      return false;
+  }
+}
+
 
 // ===================================================================
 
@@ -809,204 +1559,200 @@ void UserProperties::Swap(UserProperties* other) {
 // ===================================================================
 
 #ifndef _MSC_VER
-const int UserToBitHandshake::kChannelFieldNumber;
-const int UserToBitHandshake::kSupportListeningFieldNumber;
-const int UserToBitHandshake::kRpcVersionFieldNumber;
-const int UserToBitHandshake::kCredentialsFieldNumber;
-const int UserToBitHandshake::kPropertiesFieldNumber;
-const int UserToBitHandshake::kSupportComplexTypesFieldNumber;
-const int UserToBitHandshake::kSupportTimeoutFieldNumber;
+const int RpcEndpointInfos::kNameFieldNumber;
+const int RpcEndpointInfos::kVersionFieldNumber;
+const int RpcEndpointInfos::kMajorVersionFieldNumber;
+const int RpcEndpointInfos::kMinorVersionFieldNumber;
+const int RpcEndpointInfos::kPatchVersionFieldNumber;
+const int RpcEndpointInfos::kApplicationFieldNumber;
 #endif  // !_MSC_VER
 
-UserToBitHandshake::UserToBitHandshake()
+RpcEndpointInfos::RpcEndpointInfos()
   : ::google::protobuf::Message() {
   SharedCtor();
 }
 
-void UserToBitHandshake::InitAsDefaultInstance() {
-  credentials_ = const_cast< ::exec::shared::UserCredentials*>(&::exec::shared::UserCredentials::default_instance());
-  properties_ = const_cast< ::exec::user::UserProperties*>(&::exec::user::UserProperties::default_instance());
+void RpcEndpointInfos::InitAsDefaultInstance() {
 }
 
-UserToBitHandshake::UserToBitHandshake(const UserToBitHandshake& from)
+RpcEndpointInfos::RpcEndpointInfos(const RpcEndpointInfos& from)
   : ::google::protobuf::Message() {
   SharedCtor();
   MergeFrom(from);
 }
 
-void UserToBitHandshake::SharedCtor() {
+void RpcEndpointInfos::SharedCtor() {
   _cached_size_ = 0;
-  channel_ = 2;
-  support_listening_ = false;
-  rpc_version_ = 0;
-  credentials_ = NULL;
-  properties_ = NULL;
-  support_complex_types_ = false;
-  support_timeout_ = false;
+  name_ = const_cast< ::std::string*>(&::google::protobuf::internal::kEmptyString);
+  version_ = const_cast< ::std::string*>(&::google::protobuf::internal::kEmptyString);
+  majorversion_ = 0u;
+  minorversion_ = 0u;
+  patchversion_ = 0u;
+  application_ = const_cast< ::std::string*>(&::google::protobuf::internal::kEmptyString);
   ::memset(_has_bits_, 0, sizeof(_has_bits_));
 }
 
-UserToBitHandshake::~UserToBitHandshake() {
+RpcEndpointInfos::~RpcEndpointInfos() {
   SharedDtor();
 }
 
-void UserToBitHandshake::SharedDtor() {
+void RpcEndpointInfos::SharedDtor() {
+  if (name_ != &::google::protobuf::internal::kEmptyString) {
+    delete name_;
+  }
+  if (version_ != &::google::protobuf::internal::kEmptyString) {
+    delete version_;
+  }
+  if (application_ != &::google::protobuf::internal::kEmptyString) {
+    delete application_;
+  }
   if (this != default_instance_) {
-    delete credentials_;
-    delete properties_;
   }
 }
 
-void UserToBitHandshake::SetCachedSize(int size) const {
+void RpcEndpointInfos::SetCachedSize(int size) const {
   GOOGLE_SAFE_CONCURRENT_WRITES_BEGIN();
   _cached_size_ = size;
   GOOGLE_SAFE_CONCURRENT_WRITES_END();
 }
-const ::google::protobuf::Descriptor* UserToBitHandshake::descriptor() {
+const ::google::protobuf::Descriptor* RpcEndpointInfos::descriptor() {
   protobuf_AssignDescriptorsOnce();
-  return UserToBitHandshake_descriptor_;
+  return RpcEndpointInfos_descriptor_;
 }
 
-const UserToBitHandshake& UserToBitHandshake::default_instance() {
+const RpcEndpointInfos& RpcEndpointInfos::default_instance() {
   if (default_instance_ == NULL) protobuf_AddDesc_User_2eproto();
   return *default_instance_;
 }
 
-UserToBitHandshake* UserToBitHandshake::default_instance_ = NULL;
+RpcEndpointInfos* RpcEndpointInfos::default_instance_ = NULL;
 
-UserToBitHandshake* UserToBitHandshake::New() const {
-  return new UserToBitHandshake;
+RpcEndpointInfos* RpcEndpointInfos::New() const {
+  return new RpcEndpointInfos;
 }
 
-void UserToBitHandshake::Clear() {
+void RpcEndpointInfos::Clear() {
   if (_has_bits_[0 / 32] & (0xffu << (0 % 32))) {
-    channel_ = 2;
-    support_listening_ = false;
-    rpc_version_ = 0;
-    if (has_credentials()) {
-      if (credentials_ != NULL) credentials_->::exec::shared::UserCredentials::Clear();
+    if (has_name()) {
+      if (name_ != &::google::protobuf::internal::kEmptyString) {
+        name_->clear();
+      }
     }
-    if (has_properties()) {
-      if (properties_ != NULL) properties_->::exec::user::UserProperties::Clear();
+    if (has_version()) {
+      if (version_ != &::google::protobuf::internal::kEmptyString) {
+        version_->clear();
+      }
+    }
+    majorversion_ = 0u;
+    minorversion_ = 0u;
+    patchversion_ = 0u;
+    if (has_application()) {
+      if (application_ != &::google::protobuf::internal::kEmptyString) {
+        application_->clear();
+      }
     }
-    support_complex_types_ = false;
-    support_timeout_ = false;
   }
   ::memset(_has_bits_, 0, sizeof(_has_bits_));
   mutable_unknown_fields()->Clear();
 }
 
-bool UserToBitHandshake::MergePartialFromCodedStream(
+bool RpcEndpointInfos::MergePartialFromCodedStream(
     ::google::protobuf::io::CodedInputStream* input) {
 #define DO_(EXPRESSION) if (!(EXPRESSION)) return false
   ::google::protobuf::uint32 tag;
   while ((tag = input->ReadTag()) != 0) {
     switch (::google::protobuf::internal::WireFormatLite::GetTagFieldNumber(tag)) {
-      // optional .exec.shared.RpcChannel channel = 1 [default = USER];
+      // optional string name = 1;
       case 1: {
         if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
-            ::google::protobuf::internal::WireFormatLite::WIRETYPE_VARINT) {
-          int value;
-          DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive<
-                   int, ::google::protobuf::internal::WireFormatLite::TYPE_ENUM>(
-                 input, &value)));
-          if (::exec::shared::RpcChannel_IsValid(value)) {
-            set_channel(static_cast< ::exec::shared::RpcChannel >(value));
-          } else {
-            mutable_unknown_fields()->AddVarint(1, value);
-          }
+            ::google::protobuf::internal::WireFormatLite::WIRETYPE_LENGTH_DELIMITED) {
+          DO_(::google::protobuf::internal::WireFormatLite::ReadString(
+                input, this->mutable_name()));
+          ::google::protobuf::internal::WireFormat::VerifyUTF8String(
+            this->name().data(), this->name().length(),
+            ::google::protobuf::internal::WireFormat::PARSE);
         } else {
           goto handle_uninterpreted;
         }
-        if (input->ExpectTag(16)) goto parse_support_listening;
+        if (input->ExpectTag(18)) goto parse_version;
         break;
       }
 
-      // optional bool support_listening = 2;
+      // optional string version = 2;
       case 2: {
         if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
-            ::google::protobuf::internal::WireFormatLite::WIRETYPE_VARINT) {
-         parse_support_listening:
-          DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive<
-                   bool, ::google::protobuf::internal::WireFormatLite::TYPE_BOOL>(
-                 input, &support_listening_)));
-          set_has_support_listening();
+            ::google::protobuf::internal::WireFormatLite::WIRETYPE_LENGTH_DELIMITED) {
+         parse_version:
+          DO_(::google::protobuf::internal::WireFormatLite::ReadString(
+                input, this->mutable_version()));
+          ::google::protobuf::internal::WireFormat::VerifyUTF8String(
+            this->version().data(), this->version().length(),
+            ::google::protobuf::internal::WireFormat::PARSE);
         } else {
           goto handle_uninterpreted;
         }
-        if (input->ExpectTag(24)) goto parse_rpc_version;
+        if (input->ExpectTag(24)) goto parse_majorVersion;
         break;
       }
 
-      // optional int32 rpc_version = 3;
+      // optional uint32 majorVersion = 3;
       case 3: {
         if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
             ::google::protobuf::internal::WireFormatLite::WIRETYPE_VARINT) {
-         parse_rpc_version:
+         parse_majorVersion:
           DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive<
-                   ::google::protobuf::int32, ::google::protobuf::internal::WireFormatLite::TYPE_INT32>(
-                 input, &rpc_version_)));
-          set_has_rpc_version();
+                   ::google::protobuf::uint32, ::google::protobuf::internal::WireFormatLite::TYPE_UINT32>(
+                 input, &majorversion_)));
+          set_has_majorversion();
         } else {
           goto handle_uninterpreted;
         }
-        if (input->ExpectTag(34)) goto parse_credentials;
+        if (input->ExpectTag(32)) goto parse_minorVersion;
         break;
       }
 
-      // optional .exec.shared.UserCredentials credentials = 4;
+      // optional uint32 minorVersion = 4;
       case 4: {
         if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
-            ::google::protobuf::internal::WireFormatLite::WIRETYPE_LENGTH_DELIMITED) {
-         parse_credentials:
-          DO_(::google::protobuf::internal::WireFormatLite::ReadMessageNoVirtual(
-               input, mutable_credentials()));
+            ::google::protobuf::internal::WireFormatLite::WIRETYPE_VARINT) {
+         parse_minorVersion:
+          DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive<
+                   ::google::protobuf::uint32, ::google::protobuf::internal::WireFormatLite::TYPE_UINT32>(
+                 input, &minorversion_)));
+          set_has_minorversion();
         } else {
           goto handle_uninterpreted;
         }
-        if (input->ExpectTag(42)) goto parse_properties;
+        if (input->ExpectTag(40)) goto parse_patchVersion;
         break;
       }
 
-      // optional .exec.user.UserProperties properties = 5;
+      // optional uint32 patchVersion = 5;
       case 5: {
         if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
-            ::google::protobuf::internal::WireFormatLite::WIRETYPE_LENGTH_DELIMITED) {
-         parse_properties:
-          DO_(::google::protobuf::internal::WireFormatLite::ReadMessageNoVirtual(
-               input, mutable_properties()));
-        } else {
-          goto handle_uninterpreted;
-        }
-        if (input->ExpectTag(48)) goto parse_support_complex_types;
-        break;
-      }
-
-      // optional bool support_complex_types = 6 [default = false];
-      case 6: {
-        if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
             ::google::protobuf::internal::WireFormatLite::WIRETYPE_VARINT) {
-         parse_support_complex_types:
+         parse_patchVersion:
           DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive<
-                   bool, ::google::protobuf::internal::WireFormatLite::TYPE_BOOL>(
-                 input, &support_complex_types_)));
-          set_has_support_complex_types();
+                   ::google::protobuf::uint32, ::google::protobuf::internal::WireFormatLite::TYPE_UINT32>(
+                 input, &patchversion_)));
+          set_has_patchversion();
         } else {
           goto handle_uninterpreted;
         }
-        if (input->ExpectTag(56)) goto parse_support_timeout;
+        if (input->ExpectTag(50)) goto parse_application;
         break;
       }
 
-      // optional bool support_timeout = 7 [default = false];
-      case 7: {
+      // optional string application = 6;
+      case 6: {
         if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
-            ::google::protobuf::internal::WireFormatLite::WIRETYPE_VARINT) {
-         parse_support_timeout:
-          DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive<
-                   bool, ::google::protobuf::internal::WireFormatLite::TYPE_BOOL>(
-                 input, &support_timeout_)));
-          set_has_support_timeout();
+            ::google::protobuf::internal::WireFormatLite::WIRETYPE_LENGTH_DELIMITED) {
+         parse_application:
+          DO_(::google::protobuf::internal::WireFormatLite::ReadString(
+                input, this->mutable_application()));
+          ::google::protobuf::internal::WireFormat::VerifyUTF8String(
+            this->application().data(), this->application().length(),
+            ::google::protobuf::internal::WireFormat::PARSE);
         } else {
           goto handle_uninterpreted;
         }
@@ -1030,44 +1776,48 @@ bool UserToBitHandshake::MergePartialFromCodedStream(
 #undef DO_
 }
 
-void UserToBitHandshake::SerializeWithCachedSizes(
+void RpcEndpointInfos::SerializeWithCachedSizes(
     ::google::protobuf::io::CodedOutputStream* output) const {
-  // optional .exec.shared.RpcChannel channel = 1 [default = USER];
-  if (has_channel()) {
-    ::google::protobuf::internal::WireFormatLite::WriteEnum(
-      1, this->channel(), output);
-  }
-
-  // optional bool support_listening = 2;
-  if (has_support_listening()) {
-    ::google::protobuf::internal::WireFormatLite::WriteBool(2, this->support_listening(), output);
+  // optional string name = 1;
+  if (has_name()) {
+    ::google::protobuf::internal::WireFormat::VerifyUTF8String(
+      this->name().data(), this->name().length(),
+      ::google::protobuf::internal::WireFormat::SERIALIZE);
+    ::google::protobuf::internal::WireFormatLite::WriteString(
+      1, this->name(), output);
   }
 
-  // optional int32 rpc_version = 3;
-  if (has_rpc_version()) {
-    ::google::protobuf::internal::WireFormatLite::WriteInt32(3, this->rpc_version(), output);
+  // optional string version = 2;
+  if (has_version()) {
+    ::google::protobuf::internal::WireFormat::VerifyUTF8String(
+      this->version().data(), this->version().length(),
+      ::google::protobuf::internal::WireFormat::SERIALIZE);
+    ::google::protobuf::internal::WireFormatLite::WriteString(
+      2, this->version(), output);
   }
 
-  // optional .exec.shared.UserCredentials credentials = 4;
-  if (has_credentials()) {
-    ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray(
-      4, this->credentials(), output);
+  // optional uint32 majorVersion = 3;
+  if (has_majorversion()) {
+    ::google::protobuf::internal::WireFormatLite::WriteUInt32(3, this->majorversion(), output);
   }
 
-  // optional .exec.user.UserProperties properties = 5;
-  if (has_properties()) {
-    ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray(
-      5, this->properties(), output);
+  // optional uint32 minorVersion = 4;
+  if (has_minorversion()) {
+    ::google::protobuf::internal::WireFormatLite::WriteUInt32(4, this->minorversion(), output);
   }
 
-  // optional bool support_complex_types = 6 [default = false];
-  if (has_support_complex_types()) {
-    ::google::protobuf::internal::WireFormatLite::WriteBool(6, this->support_complex_types(), output);
+  // optional uint32 patchVersion = 5;
+  if (has_patchversion()) {
+    ::google::protobuf::internal::WireFormatLite::WriteUInt32(5, this->patchversion(), output);
   }
 
-  // optional bool support_timeout = 7 [default = false];
-  if (has_support_timeout()) {
-    ::google::protobuf::internal::WireFormatLite::WriteBool(7, this->support_timeout(), output);
+  // optional string application = 6;
+  if (has_application()) {
+    ::google::protobuf::internal::WireFormat::VerifyUTF8String(
+      this->application().data(), this->application().length(),
+      ::google::protobuf::internal::WireFormat::SERIALIZE);
+    ::google::protobuf::internal::WireFormatLite::WriteString(
+      6, this->application(), output);
   }
 
   if (!unknown_fields().empty()) {
@@ -1076,46 +1826,51 @@ void UserToBitHandshake::SerializeWithCachedSizes(
   }
 }
 
-::google::protobuf::uint8* UserToBitHandshake::SerializeWithCachedSizesToArray(
+::google::protobuf::uint8* RpcEndpointInfos::SerializeWithCachedSizesToArray(
     ::google::protobuf::uint8* target) const {
-  // optional .exec.shared.RpcChannel channel = 1 [default = USER];
-  if (has_channel()) {
-    target = ::google::protobuf::internal::WireFormatLite::WriteEnumToArray(
-      1, this->channel(), target);
-  }
-
-  // optional bool support_listening = 2;
-  if (has_support_listening()) {
-    target = ::google::protobuf::internal::WireFormatLite::WriteBoolToArray(2, this->support_listening(), target);
+  // optional string name = 1;
+  if (has_name()) {
+    ::google::protobuf::internal::WireFormat::VerifyUTF8String(
+      this->name().data(), this->name().length(),
+      ::google::protobuf::internal::WireFormat::SERIALIZE);
+    target =
+      ::google::protobuf::internal::WireFormatLite::WriteStringToArray(
+        1, this->name(), target);
   }
 
-  // optional int32 rpc_version = 3;
-  if (has_rpc_version()) {
-    target = ::google::protobuf::internal::WireFormatLite::WriteInt32ToArray(3, this->rpc_version(), target);
+  // optional string version = 2;
+  if (has_version()) {
+    ::google::protobuf::internal::WireFormat::VerifyUTF8String(
+      this->version().data(), this->version().length(),
+      ::google::protobuf::internal::WireFormat::SERIALIZE);
+    target =
+      ::google::protobuf::internal::WireFormatLite::WriteStringToArray(
+        2, this->version(), target);
   }
 
-  // optional .exec.shared.UserCredentials credentials = 4;
-  if (has_credentials()) {
-    target = ::google::protobuf::internal::WireFormatLite::
-      WriteMessageNoVirtualToArray(
-        4, this->credentials(), target);
+  // optional uint32 majorVersion = 3;
+  if (has_majorversion()) {
+    target = ::google::protobuf::internal::WireFormatLite::WriteUInt32ToArray(3, this->majorversion(), target);
   }
 
-  // optional .exec.user.UserProperties properties = 5;
-  if (has_properties()) {
-    target = ::google::protobuf::internal::WireFormatLite::
-      WriteMessageNoVirtualToArray(
-        5, this->properties(), target);
+  // optional uint32 minorVersion = 4;
+  if (has_minorversion()) {
+    target = ::google::protobuf::internal::WireFormatLite::WriteUInt32ToArray(4, this->minorversion(), target);
   }
 
-  // optional bool support_complex_types = 6 [default = false];
-  if (has_support_complex_types()) {
-    target = ::google::protobuf::internal::WireFormatLite::WriteBoolToArray(6, this->support_complex_types(), target);
+  // optional uint32 patchVersion = 5;
+  if (has_patchversion()) {
+    target = ::google::protobuf::internal::WireFormatLite::WriteUInt32ToArray(5, this->patchversion(), target);
   }
 
-  // optional bool support_timeout = 7 [default = false];
-  if (has_support_timeout()) {
-    target = ::google::protobuf::internal::WireFormatLite::WriteBoolToArray(7, this->support_timeout(), target);
+  // optional string application = 6;
+  if (has_application()) {
+    ::google::protobuf::internal::WireFormat::VerifyUTF8String(
+      this->application().data(), this->application().length(),
+      ::google::protobuf::internal::WireFormat::SERIALIZE);
+    target =
+      ::google::protobuf::internal::WireFormatLite::WriteStringToArray(
+        6, this->application(), target);
   }
 
   if (!unknown_fields().empty()) {
@@ -1125,50 +1880,50 @@ void UserToBitHandshake::SerializeWithCachedSizes(
   return target;
 }
 
-int UserToBitHandshake::ByteSize() const {
+int RpcEndpointInfos::ByteSize() const {
   int total_size = 0;
 
   if (_has_bits_[0 / 32] & (0xffu << (0 % 32))) {
-    // optional .exec.shared.RpcChannel channel = 1 [default = USER];
-    if (has_channel()) {
+    // optional string name = 1;
+    if (has_name()) {
       total_size += 1 +
-        ::google::protobuf::internal::WireFormatLite::EnumSize(this->channel());
-    }
-
-    // optional bool support_listening = 2;
-    if (has_support_listening()) {
-      total_size += 1 + 1;
+        ::google::protobuf::internal::WireFormatLite::StringSize(
+          this->name());
     }
 
-    // optional int32 rpc_version = 3;
-    if (has_rpc_version()) {
+    // optional string version = 2;
+    if (has_version()) {
       total_size += 1 +
-        ::google::protobuf::internal::WireFormatLite::Int32Size(
-          this->rpc_version());
+        ::google::protobuf::internal::WireFormatLite::StringSize(
+          this->version());
     }
 
-    // optional .exec.shared.UserCredentials credentials = 4;
-    if (has_credentials()) {
+    // optional uint32 majorVersion = 3;
+    if (has_majorversion()) {
       total_size += 1 +
-        ::google::protobuf::internal::WireFormatLite::MessageSizeNoVirtual(
-          this->credentials());
+        ::google::protobuf::internal::WireFormatLite::UInt32Size(
+          this->majorversion());
     }
 
-    // optional .exec.user.UserProperties properties = 5;
-    if (has_properties()) {
+    // optional uint32 minorVersion = 4;
+    if (has_minorversion()) {
       total_size += 1 +
-        ::google::protobuf::internal::WireFormatLite::MessageSizeNoVirtual(
-          this->properties());
+        ::google::protobuf::internal::WireFormatLite::UInt32Size(
+          this->minorversion());
     }
 
-    // optional bool support_complex_types = 6 [default = false];
-    if (has_support_complex_types()) {
-      total_size += 1 + 1;
+    // optional uint32 patchVersion = 5;
+    if (has_patchversion()) {
+      total_size += 1 +
+        ::google::protobuf::internal::WireFormatLite::UInt32Size(
+          this->patchversion());
     }
 
-    // optional bool support_timeout = 7 [default = false];
-    if (has_support_timeout()) {
-      total_size += 1 + 1;
+    // optional string application = 6;
+    if (has_application()) {
+      total_size += 1 +
+        ::google::protobuf::internal::WireFormatLite::StringSize(
+          this->application());
     }
 
   }
@@ -1183,10 +1938,10 @@ int UserToBitHandshake::ByteSize() const {
   return total_size;
 }
 
-void UserToBitHandshake::MergeFrom(const ::google::protobuf::Message& from) {
+void RpcEndpointInfos::MergeFrom(const ::google::protobuf::Message& from) {
   GOOGLE_CHECK_NE(&from, this);
-  const UserToBitHandshake* source =
-    ::google::protobuf::internal::dynamic_cast_if_available<const UserToBitHandshake*>(
+  const RpcEndpointInfos* source =
+    ::google::protobuf::internal::dynamic_cast_if_available<const RpcEndpointInfos*>(
       &from);
   if (source == NULL) {
     ::google::protobuf::internal::ReflectionOps::Merge(from, this);
@@ -1195,74 +1950,8205 @@ void UserToBitHandshake::MergeFrom(const ::google::protobuf::Message& from) {
   }
 }
 
-void UserToBitHandshake::MergeFrom(const UserToBitHandshake& from) {
+void RpcEndpointInfos::MergeFrom(const RpcEndpointInfos& from) {
   GOOGLE_CHECK_NE(&from, this);
   if (from._has_bits_[0 / 32] & (0xffu << (0 % 32))) {
-    if (from.has_channel()) {
-      set_channel(from.channel());
+    if (from.has_name()) {
+      set_name(from.name());
     }
-    if (from.has_support_listening()) {
-      set_support_listening(from.support_listening());
+    if (from.has_version()) {
+      set_version(from.version());
     }
-    if (from.has_rpc_version()) {
-      set_rpc_version(from.rpc_version());
+    if (from.has_majorversion()) {
+      set_majorversion(from.majorversion());
     }
-    if (from.has_credentials()) {
-      mutable_credentials()->::exec::shared::UserCredentials::MergeFrom(from.credentials());
+    if (from.has_minorversion()) {
+      set_minorversion(from.minorversion());
     }
-    if (from.has_properties()) {
-      mutable_properties()->::exec::user::UserProperties::MergeFrom(from.properties());
+    if (from.has_patchversion()) {
+      set_patchversion(from.patchversion());
     }
-    if (from.has_support_complex_types()) {
-      set_support_complex_types(from.support_complex_types());
+    if (from.has_application()) {
+      set_application(from.application());
     }
-    if (from.has_support_timeout()) {
-      set_support_timeout(from.support_timeout());
+  }
+  mutable_unknown_fields()->MergeFrom(from.unknown_fields());
+}
+
+void RpcEndpointInfos::CopyFrom(const ::google::protobuf::Message& from) {
+  if (&from == this) return;
+  Clear();
+  MergeFrom(from);
+}
+
+void RpcEndpointInfos::CopyFrom(const RpcEndpointInfos& from) {
+  if (&from == this) return;
+  Clear();
+  MergeFrom(from);
+}
+
+bool RpcEndpointInfos::IsInitialized() const {
+
+  return true;
+}
+
+void RpcEndpointInfos::Swap(RpcEndpointInfos* other) {
+  if (other != this) {
+    std::swap(name_, other->name_);
+    std::swap(version_, other->version_);
+    std::swap(majorversion_, other->majorversion_);
+    std::swap(minorversion_, other->minorversion_);
+    std::swap(patchversion_, other->patchversion_);
+    std::swap(application_, other->application_);
+    std::swap(_has_bits_[0], other->_has_bits_[0]);
+    _unknown_fields_.Swap(&other->_unknown_fields_);
+    std::swap(_cached_size_, other->_cached_size_);
+  }
+}
+
+::google::protobuf::Metadata RpcEndpointInfos::GetMetadata() const {
+  protobuf_AssignDescriptorsOnce();
+  ::google::protobuf::Metadata metadata;
+  metadata.descriptor = RpcEndpointInfos_descriptor_;
+  metadata.reflection = RpcEndpointInfos_reflection_;
+  return metadata;
+}
+
+
+// ===================================================================
+
+#ifndef _MSC_VER
+const int UserToBitHandshake::kChannelFieldNumber;
+const int UserToBitHandshake::kSupportListeningFieldNumber;
+const int UserToBitHandshake::kRpcVersionFieldNumber;
+const int UserToBitHandshake::kCredentialsFieldNumber;
+const int UserToBitHandshake::kPropertiesFieldNumber;
+const int UserToBitHandshake::kSupportComplexTypesFieldNumber;
+const int UserToBitHandshake::kSupportTimeoutFieldNumber;
+const int UserToBitHandshake::kClientInfosFieldNumber;
+#endif  // !_MSC_VER
+
+UserToBitHandshake::UserToBitHandshake()
+  : ::google::protobuf::Message() {
+  SharedCtor();
+}
+
+void UserToBitHandshake::InitAsDefaultInstance() {
+  credentials_ = const_cast< ::exec::shared::UserCredentials*>(&::exec::shared::UserCredentials::default_instance());
+  properties_ = const_cast< ::exec::user::UserProperties*>(&::exec::user::UserProperties::default_instance());
+  client_infos_ = const_cast< ::exec::user::RpcEndpointInfos*>(&::exec::user::RpcEndpointInfos::default_instance());
+}
+
+UserToBitHandshake::UserToBitHandshake(const UserToBitHandshake& from)
+  : ::google::protobuf::Message() {
+  SharedCtor();
+  MergeFrom(from);
+}
+
+void UserToBitHandshake::SharedCtor() {
+  _cached_size_ = 0;
+  channel_ = 2;
+  support_listening_ = false;
+  rpc_version_ = 0;
+  credentials_ = NULL;
+  properties_ = NULL;
+  support_complex_types_ = false;
+  support_timeout_ = false;
+  client_infos_ = NULL;
+  ::memset(_has_bits_, 0, sizeof(_has_bits_));
+}
+
+UserToBitHandshake::~UserToBitHandshake() {
+  SharedDtor();
+}
+
+void UserToBitHandshake::SharedDtor() {
+  if (this != default_instance_) {
+    delete credentials_;
+    delete properties_;
+    delete client_infos_;
+  }
+}
+
+void UserToBitHandshake::SetCachedSize(int size) const {
+  GOOGLE_SAFE_CONCURRENT_WRITES_BEGIN();
+  _cached_size_ = size;
+  GOOGLE_SAFE_CONCURRENT_WRITES_END();
+}
+const ::google::protobuf::Descriptor* UserToBitHandshake::descriptor() {
+  protobuf_AssignDescriptorsOnce();
+  return UserToBitHandshake_descriptor_;
+}
+
+const UserToBitHandshake& UserToBitHandshake::default_instance() {
+  if (default_instance_ == NULL) protobuf_AddDesc_User_2eproto();
+  return *default_instance_;
+}
+
+UserToBitHandshake* UserToBitHandshake::default_instance_ = NULL;
+
+UserToBitHandshake* UserToBitHandshake::New() const {
+  return new UserToBitHandshake;
+}
+
+void UserToBitHandshake::Clear() {
+  if (_has_bits_[0 / 32] & (0xffu << (0 % 32))) {
+    channel_ = 2;
+    support_listening_ = false;
+    rpc_version_ = 0;
+    if (has_credentials()) {
+      if (credentials_ != NULL) credentials_->::exec::shared::UserCredentials::Clear();
+    }
+    if (has_properties()) {
+      if (properties_ != NULL) properties_->::exec::user::UserProperties::Clear();
+    }
+    support_complex_types_ = false;
+    support_timeout_ = false;
+    if (has_client_infos()) {
+      if (client_infos_ != NULL) client_infos_->::exec::user::RpcEndpointInfos::Clear();
+    }
+  }
+  ::memset(_has_bits_, 0, sizeof(_has_bits_));
+  mutable_unknown_fields()->Clear();
+}
+
+bool UserToBitHandshake::MergePartialFromCodedStream(
+    ::google::protobuf::io::CodedInputStream* input) {
+#define DO_(EXPRESSION) if (!(EXPRESSION)) return false
+  ::google::protobuf::uint32 tag;
+  while ((tag = input->ReadTag()) != 0) {
+    switch (::google::protobuf::internal::WireFormatLite::GetTagFieldNumber(tag)) {
+      // optional .exec.shared.RpcChannel channel = 1 [default = USER];
+      case 1: {
+        if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
+            ::google::protobuf::internal::WireFormatLite::WIRETYPE_VARINT) {
+          int value;
+          DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive<
+                   int, ::google::protobuf::internal::WireFormatLite::TYPE_ENUM>(
+                 input, &value)));
+          if (::exec::shared::RpcChannel_IsValid(value)) {
+            set_channel(static_cast< ::exec::shared::RpcChannel >(value));
+          } else {
+            mutable_unknown_fields()->AddVarint(1, value);
+          }
+        } else {
+          goto handle_uninterpreted;
+        }
+        if (input->ExpectTag(16)) goto parse_support_listening;
+        break;
+      }
+
+      // optional bool support_listening = 2;
+      case 2: {
+        if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
+            ::google::protobuf::internal::WireFormatLite::WIRETYPE_VARINT) {
+         parse_support_listening:
+          DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive<
+                   bool, ::google::protobuf::internal::WireFormatLite::TYPE_BOOL>(
+                 input, &support_listening_)));
+          set_has_support_listening();
+        } else {
+          goto handle_uninterpreted;
+        }
+        if (input->ExpectTag(24)) goto parse_rpc_version;
+        break;
+      }
+
+      // optional int32 rpc_version = 3;
+      case 3: {
+        if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
+            ::google::protobuf::internal::WireFormatLite::WIRETYPE_VARINT) {
+         parse_rpc_version:
+          DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive<
+                   ::google::protobuf::int32, ::google::protobuf::internal::WireFormatLite::TYPE_INT32>(
+                 input, &rpc_version_)));
+          set_has_rpc_version();
+        } else {
+          goto handle_uninterpreted;
+        }
+        if (input->ExpectTag(34)) goto parse_credentials;
+        break;
+      }
+
+      // optional .exec.shared.UserCredentials credentials = 4;
+      case 4: {
+        if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
+            ::google::protobuf::internal::WireFormatLite::WIRETYPE_LENGTH_DELIMITED) {
+         parse_credentials:
+          DO_(::google::protobuf::internal::WireFormatLite::ReadMessageNoVirtual(
+               input, mutable_credentials()));
+        } else {
+          goto handle_uninterpreted;
+        }
+        if (input->ExpectTag(42)) goto parse_properties;
+        break;
+      }
+
+      // optional .exec.user.UserProperties properties = 5;
+      case 5: {
+        if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
+            ::google::protobuf::internal::WireFormatLite::WIRETYPE_LENGTH_DELIMITED) {
+         parse_properties:
+          DO_(::google::protobuf::internal::WireFormatLite::ReadMessageNoVirtual(
+               input, mutable_properties()));
+        } else {
+          goto handle_uninterpreted;
+        }
+        if (input->ExpectTag(48)) goto parse_support_complex_types;
+        break;
+      }
+
+      // optional bool support_complex_types = 6 [default = false];
+      case 6: {
+        if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
+            ::google::protobuf::internal::WireFormatLite::WIRETYPE_VARINT) {
+         parse_support_complex_types:
+          DO_((::google::protobuf::internal::WireFormatLite::ReadPrimitive<
+                   bool, ::google::protobuf::internal::WireFormatLite::TYPE_BOOL>(
+                 input, &support_complex_types_)));
+          set

<TRUNCATED>

[07/15] drill git commit: DRILL-4853: Update C++ protobuf source files

Posted by pa...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/2558803e/contrib/native/client/src/protobuf/User.pb.h
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/protobuf/User.pb.h b/contrib/native/client/src/protobuf/User.pb.h
index 8628a54..162c6e0 100644
--- a/contrib/native/client/src/protobuf/User.pb.h
+++ b/contrib/native/client/src/protobuf/User.pb.h
@@ -27,6 +27,9 @@
 #include <google/protobuf/unknown_field_set.h>
 #include "SchemaDef.pb.h"
 #include "UserBitShared.pb.h"
+#include "BitData.pb.h"
+#include "BitControl.pb.h"
+#include "ExecutionProtos.pb.h"
 // @@protoc_insertion_point(includes)
 
 namespace exec {
@@ -39,10 +42,31 @@ void protobuf_ShutdownFile_User_2eproto();
 
 class Property;
 class UserProperties;
+class RpcEndpointInfos;
 class UserToBitHandshake;
 class RequestResults;
-class RunQuery;
+class GetQueryPlanFragments;
+class QueryPlanFragments;
 class BitToUserHandshake;
+class LikeFilter;
+class GetCatalogsReq;
+class CatalogMetadata;
+class GetCatalogsResp;
+class GetSchemasReq;
+class SchemaMetadata;
+class GetSchemasResp;
+class GetTablesReq;
+class TableMetadata;
+class GetTablesResp;
+class GetColumnsReq;
+class ColumnMetadata;
+class GetColumnsResp;
+class CreatePreparedStatementReq;
+class ResultColumnMetadata;
+class PreparedStatementHandle;
+class PreparedStatement;
+class CreatePreparedStatementResp;
+class RunQuery;
 
 enum RpcType {
   HANDSHAKE = 0,
@@ -52,15 +76,27 @@ enum RpcType {
   CANCEL_QUERY = 4,
   REQUEST_RESULTS = 5,
   RESUME_PAUSED_QUERY = 11,
+  GET_QUERY_PLAN_FRAGMENTS = 12,
+  GET_CATALOGS = 14,
+  GET_SCHEMAS = 15,
+  GET_TABLES = 16,
+  GET_COLUMNS = 17,
+  CREATE_PREPARED_STATEMENT = 22,
   QUERY_DATA = 6,
   QUERY_HANDLE = 7,
+  QUERY_PLAN_FRAGMENTS = 13,
+  CATALOGS = 18,
+  SCHEMAS = 19,
+  TABLES = 20,
+  COLUMNS = 21,
+  PREPARED_STATEMENT = 23,
   REQ_META_FUNCTIONS = 8,
   RESP_FUNCTION_LIST = 9,
   QUERY_RESULT = 10
 };
 bool RpcType_IsValid(int value);
 const RpcType RpcType_MIN = HANDSHAKE;
-const RpcType RpcType_MAX = RESUME_PAUSED_QUERY;
+const RpcType RpcType_MAX = PREPARED_STATEMENT;
 const int RpcType_ARRAYSIZE = RpcType_MAX + 1;
 
 const ::google::protobuf::EnumDescriptor* RpcType_descriptor();
@@ -112,6 +148,69 @@ inline bool HandshakeStatus_Parse(
   return ::google::protobuf::internal::ParseNamedEnum<HandshakeStatus>(
     HandshakeStatus_descriptor(), name, value);
 }
+enum RequestStatus {
+  UNKNOWN_STATUS = 0,
+  OK = 1,
+  FAILED = 2,
+  TIMEOUT = 3
+};
+bool RequestStatus_IsValid(int value);
+const RequestStatus RequestStatus_MIN = UNKNOWN_STATUS;
+const RequestStatus RequestStatus_MAX = TIMEOUT;
+const int RequestStatus_ARRAYSIZE = RequestStatus_MAX + 1;
+
+const ::google::protobuf::EnumDescriptor* RequestStatus_descriptor();
+inline const ::std::string& RequestStatus_Name(RequestStatus value) {
+  return ::google::protobuf::internal::NameOfEnum(
+    RequestStatus_descriptor(), value);
+}
+inline bool RequestStatus_Parse(
+    const ::std::string& name, RequestStatus* value) {
+  return ::google::protobuf::internal::ParseNamedEnum<RequestStatus>(
+    RequestStatus_descriptor(), name, value);
+}
+enum ColumnSearchability {
+  UNKNOWN_SEARCHABILITY = 0,
+  NONE = 1,
+  CHAR = 2,
+  NUMBER = 3,
+  ALL = 4
+};
+bool ColumnSearchability_IsValid(int value);
+const ColumnSearchability ColumnSearchability_MIN = UNKNOWN_SEARCHABILITY;
+const ColumnSearchability ColumnSearchability_MAX = ALL;
+const int ColumnSearchability_ARRAYSIZE = ColumnSearchability_MAX + 1;
+
+const ::google::protobuf::EnumDescriptor* ColumnSearchability_descriptor();
+inline const ::std::string& ColumnSearchability_Name(ColumnSearchability value) {
+  return ::google::protobuf::internal::NameOfEnum(
+    ColumnSearchability_descriptor(), value);
+}
+inline bool ColumnSearchability_Parse(
+    const ::std::string& name, ColumnSearchability* value) {
+  return ::google::protobuf::internal::ParseNamedEnum<ColumnSearchability>(
+    ColumnSearchability_descriptor(), name, value);
+}
+enum ColumnUpdatability {
+  UNKNOWN_UPDATABILITY = 0,
+  READ_ONLY = 1,
+  WRITABLE = 2
+};
+bool ColumnUpdatability_IsValid(int value);
+const ColumnUpdatability ColumnUpdatability_MIN = UNKNOWN_UPDATABILITY;
+const ColumnUpdatability ColumnUpdatability_MAX = WRITABLE;
+const int ColumnUpdatability_ARRAYSIZE = ColumnUpdatability_MAX + 1;
+
+const ::google::protobuf::EnumDescriptor* ColumnUpdatability_descriptor();
+inline const ::std::string& ColumnUpdatability_Name(ColumnUpdatability value) {
+  return ::google::protobuf::internal::NameOfEnum(
+    ColumnUpdatability_descriptor(), value);
+}
+inline bool ColumnUpdatability_Parse(
+    const ::std::string& name, ColumnUpdatability* value) {
+  return ::google::protobuf::internal::ParseNamedEnum<ColumnUpdatability>(
+    ColumnUpdatability_descriptor(), name, value);
+}
 // ===================================================================
 
 class Property : public ::google::protobuf::Message {
@@ -301,6 +400,153 @@ class UserProperties : public ::google::protobuf::Message {
 };
 // -------------------------------------------------------------------
 
+class RpcEndpointInfos : public ::google::protobuf::Message {
+ public:
+  RpcEndpointInfos();
+  virtual ~RpcEndpointInfos();
+
+  RpcEndpointInfos(const RpcEndpointInfos& from);
+
+  inline RpcEndpointInfos& operator=(const RpcEndpointInfos& from) {
+    CopyFrom(from);
+    return *this;
+  }
+
+  inline const ::google::protobuf::UnknownFieldSet& unknown_fields() const {
+    return _unknown_fields_;
+  }
+
+  inline ::google::protobuf::UnknownFieldSet* mutable_unknown_fields() {
+    return &_unknown_fields_;
+  }
+
+  static const ::google::protobuf::Descriptor* descriptor();
+  static const RpcEndpointInfos& default_instance();
+
+  void Swap(RpcEndpointInfos* other);
+
+  // implements Message ----------------------------------------------
+
+  RpcEndpointInfos* New() const;
+  void CopyFrom(const ::google::protobuf::Message& from);
+  void MergeFrom(const ::google::protobuf::Message& from);
+  void CopyFrom(const RpcEndpointInfos& from);
+  void MergeFrom(const RpcEndpointInfos& from);
+  void Clear();
+  bool IsInitialized() const;
+
+  int ByteSize() const;
+  bool MergePartialFromCodedStream(
+      ::google::protobuf::io::CodedInputStream* input);
+  void SerializeWithCachedSizes(
+      ::google::protobuf::io::CodedOutputStream* output) const;
+  ::google::protobuf::uint8* SerializeWithCachedSizesToArray(::google::protobuf::uint8* output) const;
+  int GetCachedSize() const { return _cached_size_; }
+  private:
+  void SharedCtor();
+  void SharedDtor();
+  void SetCachedSize(int size) const;
+  public:
+
+  ::google::protobuf::Metadata GetMetadata() const;
+
+  // nested types ----------------------------------------------------
+
+  // accessors -------------------------------------------------------
+
+  // optional string name = 1;
+  inline bool has_name() const;
+  inline void clear_name();
+  static const int kNameFieldNumber = 1;
+  inline const ::std::string& name() const;
+  inline void set_name(const ::std::string& value);
+  inline void set_name(const char* value);
+  inline void set_name(const char* value, size_t size);
+  inline ::std::string* mutable_name();
+  inline ::std::string* release_name();
+  inline void set_allocated_name(::std::string* name);
+
+  // optional string version = 2;
+  inline bool has_version() const;
+  inline void clear_version();
+  static const int kVersionFieldNumber = 2;
+  inline const ::std::string& version() const;
+  inline void set_version(const ::std::string& value);
+  inline void set_version(const char* value);
+  inline void set_version(const char* value, size_t size);
+  inline ::std::string* mutable_version();
+  inline ::std::string* release_version();
+  inline void set_allocated_version(::std::string* version);
+
+  // optional uint32 majorVersion = 3;
+  inline bool has_majorversion() const;
+  inline void clear_majorversion();
+  static const int kMajorVersionFieldNumber = 3;
+  inline ::google::protobuf::uint32 majorversion() const;
+  inline void set_majorversion(::google::protobuf::uint32 value);
+
+  // optional uint32 minorVersion = 4;
+  inline bool has_minorversion() const;
+  inline void clear_minorversion();
+  static const int kMinorVersionFieldNumber = 4;
+  inline ::google::protobuf::uint32 minorversion() const;
+  inline void set_minorversion(::google::protobuf::uint32 value);
+
+  // optional uint32 patchVersion = 5;
+  inline bool has_patchversion() const;
+  inline void clear_patchversion();
+  static const int kPatchVersionFieldNumber = 5;
+  inline ::google::protobuf::uint32 patchversion() const;
+  inline void set_patchversion(::google::protobuf::uint32 value);
+
+  // optional string application = 6;
+  inline bool has_application() const;
+  inline void clear_application();
+  static const int kApplicationFieldNumber = 6;
+  inline const ::std::string& application() const;
+  inline void set_application(const ::std::string& value);
+  inline void set_application(const char* value);
+  inline void set_application(const char* value, size_t size);
+  inline ::std::string* mutable_application();
+  inline ::std::string* release_application();
+  inline void set_allocated_application(::std::string* application);
+
+  // @@protoc_insertion_point(class_scope:exec.user.RpcEndpointInfos)
+ private:
+  inline void set_has_name();
+  inline void clear_has_name();
+  inline void set_has_version();
+  inline void clear_has_version();
+  inline void set_has_majorversion();
+  inline void clear_has_majorversion();
+  inline void set_has_minorversion();
+  inline void clear_has_minorversion();
+  inline void set_has_patchversion();
+  inline void clear_has_patchversion();
+  inline void set_has_application();
+  inline void clear_has_application();
+
+  ::google::protobuf::UnknownFieldSet _unknown_fields_;
+
+  ::std::string* name_;
+  ::std::string* version_;
+  ::google::protobuf::uint32 majorversion_;
+  ::google::protobuf::uint32 minorversion_;
+  ::std::string* application_;
+  ::google::protobuf::uint32 patchversion_;
+
+  mutable int _cached_size_;
+  ::google::protobuf::uint32 _has_bits_[(6 + 31) / 32];
+
+  friend void  protobuf_AddDesc_User_2eproto();
+  friend void protobuf_AssignDesc_User_2eproto();
+  friend void protobuf_ShutdownFile_User_2eproto();
+
+  void InitAsDefaultInstance();
+  static RpcEndpointInfos* default_instance_;
+};
+// -------------------------------------------------------------------
+
 class UserToBitHandshake : public ::google::protobuf::Message {
  public:
   UserToBitHandshake();
@@ -408,6 +654,15 @@ class UserToBitHandshake : public ::google::protobuf::Message {
   inline bool support_timeout() const;
   inline void set_support_timeout(bool value);
 
+  // optional .exec.user.RpcEndpointInfos client_infos = 8;
+  inline bool has_client_infos() const;
+  inline void clear_client_infos();
+  static const int kClientInfosFieldNumber = 8;
+  inline const ::exec::user::RpcEndpointInfos& client_infos() const;
+  inline ::exec::user::RpcEndpointInfos* mutable_client_infos();
+  inline ::exec::user::RpcEndpointInfos* release_client_infos();
+  inline void set_allocated_client_infos(::exec::user::RpcEndpointInfos* client_infos);
+
   // @@protoc_insertion_point(class_scope:exec.user.UserToBitHandshake)
  private:
   inline void set_has_channel();
@@ -424,6 +679,8 @@ class UserToBitHandshake : public ::google::protobuf::Message {
   inline void clear_has_support_complex_types();
   inline void set_has_support_timeout();
   inline void clear_has_support_timeout();
+  inline void set_has_client_infos();
+  inline void clear_has_client_infos();
 
   ::google::protobuf::UnknownFieldSet _unknown_fields_;
 
@@ -431,12 +688,13 @@ class UserToBitHandshake : public ::google::protobuf::Message {
   ::google::protobuf::int32 rpc_version_;
   ::exec::shared::UserCredentials* credentials_;
   ::exec::user::UserProperties* properties_;
+  ::exec::user::RpcEndpointInfos* client_infos_;
   bool support_listening_;
   bool support_complex_types_;
   bool support_timeout_;
 
   mutable int _cached_size_;
-  ::google::protobuf::uint32 _has_bits_[(7 + 31) / 32];
+  ::google::protobuf::uint32 _has_bits_[(8 + 31) / 32];
 
   friend void  protobuf_AddDesc_User_2eproto();
   friend void protobuf_AssignDesc_User_2eproto();
@@ -541,14 +799,14 @@ class RequestResults : public ::google::protobuf::Message {
 };
 // -------------------------------------------------------------------
 
-class RunQuery : public ::google::protobuf::Message {
+class GetQueryPlanFragments : public ::google::protobuf::Message {
  public:
-  RunQuery();
-  virtual ~RunQuery();
+  GetQueryPlanFragments();
+  virtual ~GetQueryPlanFragments();
 
-  RunQuery(const RunQuery& from);
+  GetQueryPlanFragments(const GetQueryPlanFragments& from);
 
-  inline RunQuery& operator=(const RunQuery& from) {
+  inline GetQueryPlanFragments& operator=(const GetQueryPlanFragments& from) {
     CopyFrom(from);
     return *this;
   }
@@ -562,17 +820,17 @@ class RunQuery : public ::google::protobuf::Message {
   }
 
   static const ::google::protobuf::Descriptor* descriptor();
-  static const RunQuery& default_instance();
+  static const GetQueryPlanFragments& default_instance();
 
-  void Swap(RunQuery* other);
+  void Swap(GetQueryPlanFragments* other);
 
   // implements Message ----------------------------------------------
 
-  RunQuery* New() const;
+  GetQueryPlanFragments* New() const;
   void CopyFrom(const ::google::protobuf::Message& from);
   void MergeFrom(const ::google::protobuf::Message& from);
-  void CopyFrom(const RunQuery& from);
-  void MergeFrom(const RunQuery& from);
+  void CopyFrom(const GetQueryPlanFragments& from);
+  void MergeFrom(const GetQueryPlanFragments& from);
   void Clear();
   bool IsInitialized() const;
 
@@ -595,12 +853,17 @@ class RunQuery : public ::google::protobuf::Message {
 
   // accessors -------------------------------------------------------
 
-  // optional .exec.user.QueryResultsMode results_mode = 1;
-  inline bool has_results_mode() const;
-  inline void clear_results_mode();
-  static const int kResultsModeFieldNumber = 1;
-  inline ::exec::user::QueryResultsMode results_mode() const;
-  inline void set_results_mode(::exec::user::QueryResultsMode value);
+  // required string query = 1;
+  inline bool has_query() const;
+  inline void clear_query();
+  static const int kQueryFieldNumber = 1;
+  inline const ::std::string& query() const;
+  inline void set_query(const ::std::string& value);
+  inline void set_query(const char* value);
+  inline void set_query(const char* value, size_t size);
+  inline ::std::string* mutable_query();
+  inline ::std::string* release_query();
+  inline void set_allocated_query(::std::string* query);
 
   // optional .exec.shared.QueryType type = 2;
   inline bool has_type() const;
@@ -609,32 +872,27 @@ class RunQuery : public ::google::protobuf::Message {
   inline ::exec::shared::QueryType type() const;
   inline void set_type(::exec::shared::QueryType value);
 
-  // optional string plan = 3;
-  inline bool has_plan() const;
-  inline void clear_plan();
-  static const int kPlanFieldNumber = 3;
-  inline const ::std::string& plan() const;
-  inline void set_plan(const ::std::string& value);
-  inline void set_plan(const char* value);
-  inline void set_plan(const char* value, size_t size);
-  inline ::std::string* mutable_plan();
-  inline ::std::string* release_plan();
-  inline void set_allocated_plan(::std::string* plan);
+  // optional bool split_plan = 3 [default = false];
+  inline bool has_split_plan() const;
+  inline void clear_split_plan();
+  static const int kSplitPlanFieldNumber = 3;
+  inline bool split_plan() const;
+  inline void set_split_plan(bool value);
 
-  // @@protoc_insertion_point(class_scope:exec.user.RunQuery)
+  // @@protoc_insertion_point(class_scope:exec.user.GetQueryPlanFragments)
  private:
-  inline void set_has_results_mode();
-  inline void clear_has_results_mode();
+  inline void set_has_query();
+  inline void clear_has_query();
   inline void set_has_type();
   inline void clear_has_type();
-  inline void set_has_plan();
-  inline void clear_has_plan();
+  inline void set_has_split_plan();
+  inline void clear_has_split_plan();
 
   ::google::protobuf::UnknownFieldSet _unknown_fields_;
 
-  int results_mode_;
+  ::std::string* query_;
   int type_;
-  ::std::string* plan_;
+  bool split_plan_;
 
   mutable int _cached_size_;
   ::google::protobuf::uint32 _has_bits_[(3 + 31) / 32];
@@ -644,7 +902,126 @@ class RunQuery : public ::google::protobuf::Message {
   friend void protobuf_ShutdownFile_User_2eproto();
 
   void InitAsDefaultInstance();
-  static RunQuery* default_instance_;
+  static GetQueryPlanFragments* default_instance_;
+};
+// -------------------------------------------------------------------
+
+class QueryPlanFragments : public ::google::protobuf::Message {
+ public:
+  QueryPlanFragments();
+  virtual ~QueryPlanFragments();
+
+  QueryPlanFragments(const QueryPlanFragments& from);
+
+  inline QueryPlanFragments& operator=(const QueryPlanFragments& from) {
+    CopyFrom(from);
+    return *this;
+  }
+
+  inline const ::google::protobuf::UnknownFieldSet& unknown_fields() const {
+    return _unknown_fields_;
+  }
+
+  inline ::google::protobuf::UnknownFieldSet* mutable_unknown_fields() {
+    return &_unknown_fields_;
+  }
+
+  static const ::google::protobuf::Descriptor* descriptor();
+  static const QueryPlanFragments& default_instance();
+
+  void Swap(QueryPlanFragments* other);
+
+  // implements Message ----------------------------------------------
+
+  QueryPlanFragments* New() const;
+  void CopyFrom(const ::google::protobuf::Message& from);
+  void MergeFrom(const ::google::protobuf::Message& from);
+  void CopyFrom(const QueryPlanFragments& from);
+  void MergeFrom(const QueryPlanFragments& from);
+  void Clear();
+  bool IsInitialized() const;
+
+  int ByteSize() const;
+  bool MergePartialFromCodedStream(
+      ::google::protobuf::io::CodedInputStream* input);
+  void SerializeWithCachedSizes(
+      ::google::protobuf::io::CodedOutputStream* output) const;
+  ::google::protobuf::uint8* SerializeWithCachedSizesToArray(::google::protobuf::uint8* output) const;
+  int GetCachedSize() const { return _cached_size_; }
+  private:
+  void SharedCtor();
+  void SharedDtor();
+  void SetCachedSize(int size) const;
+  public:
+
+  ::google::protobuf::Metadata GetMetadata() const;
+
+  // nested types ----------------------------------------------------
+
+  // accessors -------------------------------------------------------
+
+  // required .exec.shared.QueryResult.QueryState status = 1;
+  inline bool has_status() const;
+  inline void clear_status();
+  static const int kStatusFieldNumber = 1;
+  inline ::exec::shared::QueryResult_QueryState status() const;
+  inline void set_status(::exec::shared::QueryResult_QueryState value);
+
+  // optional .exec.shared.QueryId query_id = 2;
+  inline bool has_query_id() const;
+  inline void clear_query_id();
+  static const int kQueryIdFieldNumber = 2;
+  inline const ::exec::shared::QueryId& query_id() const;
+  inline ::exec::shared::QueryId* mutable_query_id();
+  inline ::exec::shared::QueryId* release_query_id();
+  inline void set_allocated_query_id(::exec::shared::QueryId* query_id);
+
+  // repeated .exec.bit.control.PlanFragment fragments = 3;
+  inline int fragments_size() const;
+  inline void clear_fragments();
+  static const int kFragmentsFieldNumber = 3;
+  inline const ::exec::bit::control::PlanFragment& fragments(int index) const;
+  inline ::exec::bit::control::PlanFragment* mutable_fragments(int index);
+  inline ::exec::bit::control::PlanFragment* add_fragments();
+  inline const ::google::protobuf::RepeatedPtrField< ::exec::bit::control::PlanFragment >&
+      fragments() const;
+  inline ::google::protobuf::RepeatedPtrField< ::exec::bit::control::PlanFragment >*
+      mutable_fragments();
+
+  // optional .exec.shared.DrillPBError error = 4;
+  inline bool has_error() const;
+  inline void clear_error();
+  static const int kErrorFieldNumber = 4;
+  inline const ::exec::shared::DrillPBError& error() const;
+  inline ::exec::shared::DrillPBError* mutable_error();
+  inline ::exec::shared::DrillPBError* release_error();
+  inline void set_allocated_error(::exec::shared::DrillPBError* error);
+
+  // @@protoc_insertion_point(class_scope:exec.user.QueryPlanFragments)
+ private:
+  inline void set_has_status();
+  inline void clear_has_status();
+  inline void set_has_query_id();
+  inline void clear_has_query_id();
+  inline void set_has_error();
+  inline void clear_has_error();
+
+  ::google::protobuf::UnknownFieldSet _unknown_fields_;
+
+  ::exec::shared::QueryId* query_id_;
+  ::google::protobuf::RepeatedPtrField< ::exec::bit::control::PlanFragment > fragments_;
+  ::exec::shared::DrillPBError* error_;
+  int status_;
+
+  mutable int _cached_size_;
+  ::google::protobuf::uint32 _has_bits_[(4 + 31) / 32];
+
+  friend void  protobuf_AddDesc_User_2eproto();
+  friend void protobuf_AssignDesc_User_2eproto();
+  friend void protobuf_ShutdownFile_User_2eproto();
+
+  void InitAsDefaultInstance();
+  static QueryPlanFragments* default_instance_;
 };
 // -------------------------------------------------------------------
 
@@ -740,6 +1117,15 @@ class BitToUserHandshake : public ::google::protobuf::Message {
   inline ::std::string* release_errormessage();
   inline void set_allocated_errormessage(::std::string* errormessage);
 
+  // optional .exec.user.RpcEndpointInfos server_infos = 6;
+  inline bool has_server_infos() const;
+  inline void clear_server_infos();
+  static const int kServerInfosFieldNumber = 6;
+  inline const ::exec::user::RpcEndpointInfos& server_infos() const;
+  inline ::exec::user::RpcEndpointInfos* mutable_server_infos();
+  inline ::exec::user::RpcEndpointInfos* release_server_infos();
+  inline void set_allocated_server_infos(::exec::user::RpcEndpointInfos* server_infos);
+
   // @@protoc_insertion_point(class_scope:exec.user.BitToUserHandshake)
  private:
   inline void set_has_rpc_version();
@@ -750,6 +1136,8 @@ class BitToUserHandshake : public ::google::protobuf::Message {
   inline void clear_has_errorid();
   inline void set_has_errormessage();
   inline void clear_has_errormessage();
+  inline void set_has_server_infos();
+  inline void clear_has_server_infos();
 
   ::google::protobuf::UnknownFieldSet _unknown_fields_;
 
@@ -757,9 +1145,10 @@ class BitToUserHandshake : public ::google::protobuf::Message {
   int status_;
   ::std::string* errorid_;
   ::std::string* errormessage_;
+  ::exec::user::RpcEndpointInfos* server_infos_;
 
   mutable int _cached_size_;
-  ::google::protobuf::uint32 _has_bits_[(4 + 31) / 32];
+  ::google::protobuf::uint32 _has_bits_[(5 + 31) / 32];
 
   friend void  protobuf_AddDesc_User_2eproto();
   friend void protobuf_AssignDesc_User_2eproto();
@@ -768,435 +1157,7249 @@ class BitToUserHandshake : public ::google::protobuf::Message {
   void InitAsDefaultInstance();
   static BitToUserHandshake* default_instance_;
 };
-// ===================================================================
-
+// -------------------------------------------------------------------
 
-// ===================================================================
+class LikeFilter : public ::google::protobuf::Message {
+ public:
+  LikeFilter();
+  virtual ~LikeFilter();
 
-// Property
+  LikeFilter(const LikeFilter& from);
 
-// required string key = 1;
-inline bool Property::has_key() const {
-  return (_has_bits_[0] & 0x00000001u) != 0;
-}
-inline void Property::set_has_key() {
-  _has_bits_[0] |= 0x00000001u;
-}
-inline void Property::clear_has_key() {
-  _has_bits_[0] &= ~0x00000001u;
-}
-inline void Property::clear_key() {
-  if (key_ != &::google::protobuf::internal::kEmptyString) {
-    key_->clear();
-  }
-  clear_has_key();
-}
-inline const ::std::string& Property::key() const {
-  return *key_;
-}
-inline void Property::set_key(const ::std::string& value) {
-  set_has_key();
-  if (key_ == &::google::protobuf::internal::kEmptyString) {
-    key_ = new ::std::string;
-  }
-  key_->assign(value);
-}
-inline void Property::set_key(const char* value) {
-  set_has_key();
-  if (key_ == &::google::protobuf::internal::kEmptyString) {
-    key_ = new ::std::string;
-  }
-  key_->assign(value);
-}
-inline void Property::set_key(const char* value, size_t size) {
-  set_has_key();
-  if (key_ == &::google::protobuf::internal::kEmptyString) {
-    key_ = new ::std::string;
+  inline LikeFilter& operator=(const LikeFilter& from) {
+    CopyFrom(from);
+    return *this;
   }
-  key_->assign(reinterpret_cast<const char*>(value), size);
-}
-inline ::std::string* Property::mutable_key() {
-  set_has_key();
-  if (key_ == &::google::protobuf::internal::kEmptyString) {
-    key_ = new ::std::string;
+
+  inline const ::google::protobuf::UnknownFieldSet& unknown_fields() const {
+    return _unknown_fields_;
   }
-  return key_;
-}
-inline ::std::string* Property::release_key() {
-  clear_has_key();
-  if (key_ == &::google::protobuf::internal::kEmptyString) {
-    return NULL;
-  } else {
-    ::std::string* temp = key_;
-    key_ = const_cast< ::std::string*>(&::google::protobuf::internal::kEmptyString);
-    return temp;
+
+  inline ::google::protobuf::UnknownFieldSet* mutable_unknown_fields() {
+    return &_unknown_fields_;
   }
-}
+
+  static const ::google::protobuf::Descriptor* descriptor();
+  static const LikeFilter& default_instance();
+
+  void Swap(LikeFilter* other);
+
+  // implements Message ----------------------------------------------
+
+  LikeFilter* New() const;
+  void CopyFrom(const ::google::protobuf::Message& from);
+  void MergeFrom(const ::google::protobuf::Message& from);
+  void CopyFrom(const LikeFilter& from);
+  void MergeFrom(const LikeFilter& from);
+  void Clear();
+  bool IsInitialized() const;
+
+  int ByteSize() const;
+  bool MergePartialFromCodedStream(
+      ::google::protobuf::io::CodedInputStream* input);
+  void SerializeWithCachedSizes(
+      ::google::protobuf::io::CodedOutputStream* output) const;
+  ::google::protobuf::uint8* SerializeWithCachedSizesToArray(::google::protobuf::uint8* output) const;
+  int GetCachedSize() const { return _cached_size_; }
+  private:
+  void SharedCtor();
+  void SharedDtor();
+  void SetCachedSize(int size) const;
+  public:
+
+  ::google::protobuf::Metadata GetMetadata() const;
+
+  // nested types ----------------------------------------------------
+
+  // accessors -------------------------------------------------------
+
+  // optional string pattern = 1;
+  inline bool has_pattern() const;
+  inline void clear_pattern();
+  static const int kPatternFieldNumber = 1;
+  inline const ::std::string& pattern() const;
+  inline void set_pattern(const ::std::string& value);
+  inline void set_pattern(const char* value);
+  inline void set_pattern(const char* value, size_t size);
+  inline ::std::string* mutable_pattern();
+  inline ::std::string* release_pattern();
+  inline void set_allocated_pattern(::std::string* pattern);
+
+  // optional string escape = 2;
+  inline bool has_escape() const;
+  inline void clear_escape();
+  static const int kEscapeFieldNumber = 2;
+  inline const ::std::string& escape() const;
+  inline void set_escape(const ::std::string& value);
+  inline void set_escape(const char* value);
+  inline void set_escape(const char* value, size_t size);
+  inline ::std::string* mutable_escape();
+  inline ::std::string* release_escape();
+  inline void set_allocated_escape(::std::string* escape);
+
+  // @@protoc_insertion_point(class_scope:exec.user.LikeFilter)
+ private:
+  inline void set_has_pattern();
+  inline void clear_has_pattern();
+  inline void set_has_escape();
+  inline void clear_has_escape();
+
+  ::google::protobuf::UnknownFieldSet _unknown_fields_;
+
+  ::std::string* pattern_;
+  ::std::string* escape_;
+
+  mutable int _cached_size_;
+  ::google::protobuf::uint32 _has_bits_[(2 + 31) / 32];
+
+  friend void  protobuf_AddDesc_User_2eproto();
+  friend void protobuf_AssignDesc_User_2eproto();
+  friend void protobuf_ShutdownFile_User_2eproto();
+
+  void InitAsDefaultInstance();
+  static LikeFilter* default_instance_;
+};
+// -------------------------------------------------------------------
+
+class GetCatalogsReq : public ::google::protobuf::Message {
+ public:
+  GetCatalogsReq();
+  virtual ~GetCatalogsReq();
+
+  GetCatalogsReq(const GetCatalogsReq& from);
+
+  inline GetCatalogsReq& operator=(const GetCatalogsReq& from) {
+    CopyFrom(from);
+    return *this;
+  }
+
+  inline const ::google::protobuf::UnknownFieldSet& unknown_fields() const {
+    return _unknown_fields_;
+  }
+
+  inline ::google::protobuf::UnknownFieldSet* mutable_unknown_fields() {
+    return &_unknown_fields_;
+  }
+
+  static const ::google::protobuf::Descriptor* descriptor();
+  static const GetCatalogsReq& default_instance();
+
+  void Swap(GetCatalogsReq* other);
+
+  // implements Message ----------------------------------------------
+
+  GetCatalogsReq* New() const;
+  void CopyFrom(const ::google::protobuf::Message& from);
+  void MergeFrom(const ::google::protobuf::Message& from);
+  void CopyFrom(const GetCatalogsReq& from);
+  void MergeFrom(const GetCatalogsReq& from);
+  void Clear();
+  bool IsInitialized() const;
+
+  int ByteSize() const;
+  bool MergePartialFromCodedStream(
+      ::google::protobuf::io::CodedInputStream* input);
+  void SerializeWithCachedSizes(
+      ::google::protobuf::io::CodedOutputStream* output) const;
+  ::google::protobuf::uint8* SerializeWithCachedSizesToArray(::google::protobuf::uint8* output) const;
+  int GetCachedSize() const { return _cached_size_; }
+  private:
+  void SharedCtor();
+  void SharedDtor();
+  void SetCachedSize(int size) const;
+  public:
+
+  ::google::protobuf::Metadata GetMetadata() const;
+
+  // nested types ----------------------------------------------------
+
+  // accessors -------------------------------------------------------
+
+  // optional .exec.user.LikeFilter catalog_name_filter = 1;
+  inline bool has_catalog_name_filter() const;
+  inline void clear_catalog_name_filter();
+  static const int kCatalogNameFilterFieldNumber = 1;
+  inline const ::exec::user::LikeFilter& catalog_name_filter() const;
+  inline ::exec::user::LikeFilter* mutable_catalog_name_filter();
+  inline ::exec::user::LikeFilter* release_catalog_name_filter();
+  inline void set_allocated_catalog_name_filter(::exec::user::LikeFilter* catalog_name_filter);
+
+  // @@protoc_insertion_point(class_scope:exec.user.GetCatalogsReq)
+ private:
+  inline void set_has_catalog_name_filter();
+  inline void clear_has_catalog_name_filter();
+
+  ::google::protobuf::UnknownFieldSet _unknown_fields_;
+
+  ::exec::user::LikeFilter* catalog_name_filter_;
+
+  mutable int _cached_size_;
+  ::google::protobuf::uint32 _has_bits_[(1 + 31) / 32];
+
+  friend void  protobuf_AddDesc_User_2eproto();
+  friend void protobuf_AssignDesc_User_2eproto();
+  friend void protobuf_ShutdownFile_User_2eproto();
+
+  void InitAsDefaultInstance();
+  static GetCatalogsReq* default_instance_;
+};
+// -------------------------------------------------------------------
+
+class CatalogMetadata : public ::google::protobuf::Message {
+ public:
+  CatalogMetadata();
+  virtual ~CatalogMetadata();
+
+  CatalogMetadata(const CatalogMetadata& from);
+
+  inline CatalogMetadata& operator=(const CatalogMetadata& from) {
+    CopyFrom(from);
+    return *this;
+  }
+
+  inline const ::google::protobuf::UnknownFieldSet& unknown_fields() const {
+    return _unknown_fields_;
+  }
+
+  inline ::google::protobuf::UnknownFieldSet* mutable_unknown_fields() {
+    return &_unknown_fields_;
+  }
+
+  static const ::google::protobuf::Descriptor* descriptor();
+  static const CatalogMetadata& default_instance();
+
+  void Swap(CatalogMetadata* other);
+
+  // implements Message ----------------------------------------------
+
+  CatalogMetadata* New() const;
+  void CopyFrom(const ::google::protobuf::Message& from);
+  void MergeFrom(const ::google::protobuf::Message& from);
+  void CopyFrom(const CatalogMetadata& from);
+  void MergeFrom(const CatalogMetadata& from);
+  void Clear();
+  bool IsInitialized() const;
+
+  int ByteSize() const;
+  bool MergePartialFromCodedStream(
+      ::google::protobuf::io::CodedInputStream* input);
+  void SerializeWithCachedSizes(
+      ::google::protobuf::io::CodedOutputStream* output) const;
+  ::google::protobuf::uint8* SerializeWithCachedSizesToArray(::google::protobuf::uint8* output) const;
+  int GetCachedSize() const { return _cached_size_; }
+  private:
+  void SharedCtor();
+  void SharedDtor();
+  void SetCachedSize(int size) const;
+  public:
+
+  ::google::protobuf::Metadata GetMetadata() const;
+
+  // nested types ----------------------------------------------------
+
+  // accessors -------------------------------------------------------
+
+  // optional string catalog_name = 1;
+  inline bool has_catalog_name() const;
+  inline void clear_catalog_name();
+  static const int kCatalogNameFieldNumber = 1;
+  inline const ::std::string& catalog_name() const;
+  inline void set_catalog_name(const ::std::string& value);
+  inline void set_catalog_name(const char* value);
+  inline void set_catalog_name(const char* value, size_t size);
+  inline ::std::string* mutable_catalog_name();
+  inline ::std::string* release_catalog_name();
+  inline void set_allocated_catalog_name(::std::string* catalog_name);
+
+  // optional string description = 2;
+  inline bool has_description() const;
+  inline void clear_description();
+  static const int kDescriptionFieldNumber = 2;
+  inline const ::std::string& description() const;
+  inline void set_description(const ::std::string& value);
+  inline void set_description(const char* value);
+  inline void set_description(const char* value, size_t size);
+  inline ::std::string* mutable_description();
+  inline ::std::string* release_description();
+  inline void set_allocated_description(::std::string* description);
+
+  // optional string connect = 3;
+  inline bool has_connect() const;
+  inline void clear_connect();
+  static const int kConnectFieldNumber = 3;
+  inline const ::std::string& connect() const;
+  inline void set_connect(const ::std::string& value);
+  inline void set_connect(const char* value);
+  inline void set_connect(const char* value, size_t size);
+  inline ::std::string* mutable_connect();
+  inline ::std::string* release_connect();
+  inline void set_allocated_connect(::std::string* connect);
+
+  // @@protoc_insertion_point(class_scope:exec.user.CatalogMetadata)
+ private:
+  inline void set_has_catalog_name();
+  inline void clear_has_catalog_name();
+  inline void set_has_description();
+  inline void clear_has_description();
+  inline void set_has_connect();
+  inline void clear_has_connect();
+
+  ::google::protobuf::UnknownFieldSet _unknown_fields_;
+
+  ::std::string* catalog_name_;
+  ::std::string* description_;
+  ::std::string* connect_;
+
+  mutable int _cached_size_;
+  ::google::protobuf::uint32 _has_bits_[(3 + 31) / 32];
+
+  friend void  protobuf_AddDesc_User_2eproto();
+  friend void protobuf_AssignDesc_User_2eproto();
+  friend void protobuf_ShutdownFile_User_2eproto();
+
+  void InitAsDefaultInstance();
+  static CatalogMetadata* default_instance_;
+};
+// -------------------------------------------------------------------
+
+class GetCatalogsResp : public ::google::protobuf::Message {
+ public:
+  GetCatalogsResp();
+  virtual ~GetCatalogsResp();
+
+  GetCatalogsResp(const GetCatalogsResp& from);
+
+  inline GetCatalogsResp& operator=(const GetCatalogsResp& from) {
+    CopyFrom(from);
+    return *this;
+  }
+
+  inline const ::google::protobuf::UnknownFieldSet& unknown_fields() const {
+    return _unknown_fields_;
+  }
+
+  inline ::google::protobuf::UnknownFieldSet* mutable_unknown_fields() {
+    return &_unknown_fields_;
+  }
+
+  static const ::google::protobuf::Descriptor* descriptor();
+  static const GetCatalogsResp& default_instance();
+
+  void Swap(GetCatalogsResp* other);
+
+  // implements Message ----------------------------------------------
+
+  GetCatalogsResp* New() const;
+  void CopyFrom(const ::google::protobuf::Message& from);
+  void MergeFrom(const ::google::protobuf::Message& from);
+  void CopyFrom(const GetCatalogsResp& from);
+  void MergeFrom(const GetCatalogsResp& from);
+  void Clear();
+  bool IsInitialized() const;
+
+  int ByteSize() const;
+  bool MergePartialFromCodedStream(
+      ::google::protobuf::io::CodedInputStream* input);
+  void SerializeWithCachedSizes(
+      ::google::protobuf::io::CodedOutputStream* output) const;
+  ::google::protobuf::uint8* SerializeWithCachedSizesToArray(::google::protobuf::uint8* output) const;
+  int GetCachedSize() const { return _cached_size_; }
+  private:
+  void SharedCtor();
+  void SharedDtor();
+  void SetCachedSize(int size) const;
+  public:
+
+  ::google::protobuf::Metadata GetMetadata() const;
+
+  // nested types ----------------------------------------------------
+
+  // accessors -------------------------------------------------------
+
+  // optional .exec.user.RequestStatus status = 1;
+  inline bool has_status() const;
+  inline void clear_status();
+  static const int kStatusFieldNumber = 1;
+  inline ::exec::user::RequestStatus status() const;
+  inline void set_status(::exec::user::RequestStatus value);
+
+  // repeated .exec.user.CatalogMetadata catalogs = 2;
+  inline int catalogs_size() const;
+  inline void clear_catalogs();
+  static const int kCatalogsFieldNumber = 2;
+  inline const ::exec::user::CatalogMetadata& catalogs(int index) const;
+  inline ::exec::user::CatalogMetadata* mutable_catalogs(int index);
+  inline ::exec::user::CatalogMetadata* add_catalogs();
+  inline const ::google::protobuf::RepeatedPtrField< ::exec::user::CatalogMetadata >&
+      catalogs() const;
+  inline ::google::protobuf::RepeatedPtrField< ::exec::user::CatalogMetadata >*
+      mutable_catalogs();
+
+  // optional .exec.shared.DrillPBError error = 3;
+  inline bool has_error() const;
+  inline void clear_error();
+  static const int kErrorFieldNumber = 3;
+  inline const ::exec::shared::DrillPBError& error() const;
+  inline ::exec::shared::DrillPBError* mutable_error();
+  inline ::exec::shared::DrillPBError* release_error();
+  inline void set_allocated_error(::exec::shared::DrillPBError* error);
+
+  // @@protoc_insertion_point(class_scope:exec.user.GetCatalogsResp)
+ private:
+  inline void set_has_status();
+  inline void clear_has_status();
+  inline void set_has_error();
+  inline void clear_has_error();
+
+  ::google::protobuf::UnknownFieldSet _unknown_fields_;
+
+  ::google::protobuf::RepeatedPtrField< ::exec::user::CatalogMetadata > catalogs_;
+  ::exec::shared::DrillPBError* error_;
+  int status_;
+
+  mutable int _cached_size_;
+  ::google::protobuf::uint32 _has_bits_[(3 + 31) / 32];
+
+  friend void  protobuf_AddDesc_User_2eproto();
+  friend void protobuf_AssignDesc_User_2eproto();
+  friend void protobuf_ShutdownFile_User_2eproto();
+
+  void InitAsDefaultInstance();
+  static GetCatalogsResp* default_instance_;
+};
+// -------------------------------------------------------------------
+
+class GetSchemasReq : public ::google::protobuf::Message {
+ public:
+  GetSchemasReq();
+  virtual ~GetSchemasReq();
+
+  GetSchemasReq(const GetSchemasReq& from);
+
+  inline GetSchemasReq& operator=(const GetSchemasReq& from) {
+    CopyFrom(from);
+    return *this;
+  }
+
+  inline const ::google::protobuf::UnknownFieldSet& unknown_fields() const {
+    return _unknown_fields_;
+  }
+
+  inline ::google::protobuf::UnknownFieldSet* mutable_unknown_fields() {
+    return &_unknown_fields_;
+  }
+
+  static const ::google::protobuf::Descriptor* descriptor();
+  static const GetSchemasReq& default_instance();
+
+  void Swap(GetSchemasReq* other);
+
+  // implements Message ----------------------------------------------
+
+  GetSchemasReq* New() const;
+  void CopyFrom(const ::google::protobuf::Message& from);
+  void MergeFrom(const ::google::protobuf::Message& from);
+  void CopyFrom(const GetSchemasReq& from);
+  void MergeFrom(const GetSchemasReq& from);
+  void Clear();
+  bool IsInitialized() const;
+
+  int ByteSize() const;
+  bool MergePartialFromCodedStream(
+      ::google::protobuf::io::CodedInputStream* input);
+  void SerializeWithCachedSizes(
+      ::google::protobuf::io::CodedOutputStream* output) const;
+  ::google::protobuf::uint8* SerializeWithCachedSizesToArray(::google::protobuf::uint8* output) const;
+  int GetCachedSize() const { return _cached_size_; }
+  private:
+  void SharedCtor();
+  void SharedDtor();
+  void SetCachedSize(int size) const;
+  public:
+
+  ::google::protobuf::Metadata GetMetadata() const;
+
+  // nested types ----------------------------------------------------
+
+  // accessors -------------------------------------------------------
+
+  // optional .exec.user.LikeFilter catalog_name_filter = 1;
+  inline bool has_catalog_name_filter() const;
+  inline void clear_catalog_name_filter();
+  static const int kCatalogNameFilterFieldNumber = 1;
+  inline const ::exec::user::LikeFilter& catalog_name_filter() const;
+  inline ::exec::user::LikeFilter* mutable_catalog_name_filter();
+  inline ::exec::user::LikeFilter* release_catalog_name_filter();
+  inline void set_allocated_catalog_name_filter(::exec::user::LikeFilter* catalog_name_filter);
+
+  // optional .exec.user.LikeFilter schema_name_filter = 2;
+  inline bool has_schema_name_filter() const;
+  inline void clear_schema_name_filter();
+  static const int kSchemaNameFilterFieldNumber = 2;
+  inline const ::exec::user::LikeFilter& schema_name_filter() const;
+  inline ::exec::user::LikeFilter* mutable_schema_name_filter();
+  inline ::exec::user::LikeFilter* release_schema_name_filter();
+  inline void set_allocated_schema_name_filter(::exec::user::LikeFilter* schema_name_filter);
+
+  // @@protoc_insertion_point(class_scope:exec.user.GetSchemasReq)
+ private:
+  inline void set_has_catalog_name_filter();
+  inline void clear_has_catalog_name_filter();
+  inline void set_has_schema_name_filter();
+  inline void clear_has_schema_name_filter();
+
+  ::google::protobuf::UnknownFieldSet _unknown_fields_;
+
+  ::exec::user::LikeFilter* catalog_name_filter_;
+  ::exec::user::LikeFilter* schema_name_filter_;
+
+  mutable int _cached_size_;
+  ::google::protobuf::uint32 _has_bits_[(2 + 31) / 32];
+
+  friend void  protobuf_AddDesc_User_2eproto();
+  friend void protobuf_AssignDesc_User_2eproto();
+  friend void protobuf_ShutdownFile_User_2eproto();
+
+  void InitAsDefaultInstance();
+  static GetSchemasReq* default_instance_;
+};
+// -------------------------------------------------------------------
+
+class SchemaMetadata : public ::google::protobuf::Message {
+ public:
+  SchemaMetadata();
+  virtual ~SchemaMetadata();
+
+  SchemaMetadata(const SchemaMetadata& from);
+
+  inline SchemaMetadata& operator=(const SchemaMetadata& from) {
+    CopyFrom(from);
+    return *this;
+  }
+
+  inline const ::google::protobuf::UnknownFieldSet& unknown_fields() const {
+    return _unknown_fields_;
+  }
+
+  inline ::google::protobuf::UnknownFieldSet* mutable_unknown_fields() {
+    return &_unknown_fields_;
+  }
+
+  static const ::google::protobuf::Descriptor* descriptor();
+  static const SchemaMetadata& default_instance();
+
+  void Swap(SchemaMetadata* other);
+
+  // implements Message ----------------------------------------------
+
+  SchemaMetadata* New() const;
+  void CopyFrom(const ::google::protobuf::Message& from);
+  void MergeFrom(const ::google::protobuf::Message& from);
+  void CopyFrom(const SchemaMetadata& from);
+  void MergeFrom(const SchemaMetadata& from);
+  void Clear();
+  bool IsInitialized() const;
+
+  int ByteSize() const;
+  bool MergePartialFromCodedStream(
+      ::google::protobuf::io::CodedInputStream* input);
+  void SerializeWithCachedSizes(
+      ::google::protobuf::io::CodedOutputStream* output) const;
+  ::google::protobuf::uint8* SerializeWithCachedSizesToArray(::google::protobuf::uint8* output) const;
+  int GetCachedSize() const { return _cached_size_; }
+  private:
+  void SharedCtor();
+  void SharedDtor();
+  void SetCachedSize(int size) const;
+  public:
+
+  ::google::protobuf::Metadata GetMetadata() const;
+
+  // nested types ----------------------------------------------------
+
+  // accessors -------------------------------------------------------
+
+  // optional string catalog_name = 1;
+  inline bool has_catalog_name() const;
+  inline void clear_catalog_name();
+  static const int kCatalogNameFieldNumber = 1;
+  inline const ::std::string& catalog_name() const;
+  inline void set_catalog_name(const ::std::string& value);
+  inline void set_catalog_name(const char* value);
+  inline void set_catalog_name(const char* value, size_t size);
+  inline ::std::string* mutable_catalog_name();
+  inline ::std::string* release_catalog_name();
+  inline void set_allocated_catalog_name(::std::string* catalog_name);
+
+  // optional string schema_name = 2;
+  inline bool has_schema_name() const;
+  inline void clear_schema_name();
+  static const int kSchemaNameFieldNumber = 2;
+  inline const ::std::string& schema_name() const;
+  inline void set_schema_name(const ::std::string& value);
+  inline void set_schema_name(const char* value);
+  inline void set_schema_name(const char* value, size_t size);
+  inline ::std::string* mutable_schema_name();
+  inline ::std::string* release_schema_name();
+  inline void set_allocated_schema_name(::std::string* schema_name);
+
+  // optional string owner = 3;
+  inline bool has_owner() const;
+  inline void clear_owner();
+  static const int kOwnerFieldNumber = 3;
+  inline const ::std::string& owner() const;
+  inline void set_owner(const ::std::string& value);
+  inline void set_owner(const char* value);
+  inline void set_owner(const char* value, size_t size);
+  inline ::std::string* mutable_owner();
+  inline ::std::string* release_owner();
+  inline void set_allocated_owner(::std::string* owner);
+
+  // optional string type = 4;
+  inline bool has_type() const;
+  inline void clear_type();
+  static const int kTypeFieldNumber = 4;
+  inline const ::std::string& type() const;
+  inline void set_type(const ::std::string& value);
+  inline void set_type(const char* value);
+  inline void set_type(const char* value, size_t size);
+  inline ::std::string* mutable_type();
+  inline ::std::string* release_type();
+  inline void set_allocated_type(::std::string* type);
+
+  // optional string mutable = 5;
+  inline bool has_mutable_() const;
+  inline void clear_mutable_();
+  static const int kMutableFieldNumber = 5;
+  inline const ::std::string& mutable_() const;
+  inline void set_mutable_(const ::std::string& value);
+  inline void set_mutable_(const char* value);
+  inline void set_mutable_(const char* value, size_t size);
+  inline ::std::string* mutable_mutable_();
+  inline ::std::string* release_mutable_();
+  inline void set_allocated_mutable_(::std::string* mutable_);
+
+  // @@protoc_insertion_point(class_scope:exec.user.SchemaMetadata)
+ private:
+  inline void set_has_catalog_name();
+  inline void clear_has_catalog_name();
+  inline void set_has_schema_name();
+  inline void clear_has_schema_name();
+  inline void set_has_owner();
+  inline void clear_has_owner();
+  inline void set_has_type();
+  inline void clear_has_type();
+  inline void set_has_mutable_();
+  inline void clear_has_mutable_();
+
+  ::google::protobuf::UnknownFieldSet _unknown_fields_;
+
+  ::std::string* catalog_name_;
+  ::std::string* schema_name_;
+  ::std::string* owner_;
+  ::std::string* type_;
+  ::std::string* mutable__;
+
+  mutable int _cached_size_;
+  ::google::protobuf::uint32 _has_bits_[(5 + 31) / 32];
+
+  friend void  protobuf_AddDesc_User_2eproto();
+  friend void protobuf_AssignDesc_User_2eproto();
+  friend void protobuf_ShutdownFile_User_2eproto();
+
+  void InitAsDefaultInstance();
+  static SchemaMetadata* default_instance_;
+};
+// -------------------------------------------------------------------
+
+class GetSchemasResp : public ::google::protobuf::Message {
+ public:
+  GetSchemasResp();
+  virtual ~GetSchemasResp();
+
+  GetSchemasResp(const GetSchemasResp& from);
+
+  inline GetSchemasResp& operator=(const GetSchemasResp& from) {
+    CopyFrom(from);
+    return *this;
+  }
+
+  inline const ::google::protobuf::UnknownFieldSet& unknown_fields() const {
+    return _unknown_fields_;
+  }
+
+  inline ::google::protobuf::UnknownFieldSet* mutable_unknown_fields() {
+    return &_unknown_fields_;
+  }
+
+  static const ::google::protobuf::Descriptor* descriptor();
+  static const GetSchemasResp& default_instance();
+
+  void Swap(GetSchemasResp* other);
+
+  // implements Message ----------------------------------------------
+
+  GetSchemasResp* New() const;
+  void CopyFrom(const ::google::protobuf::Message& from);
+  void MergeFrom(const ::google::protobuf::Message& from);
+  void CopyFrom(const GetSchemasResp& from);
+  void MergeFrom(const GetSchemasResp& from);
+  void Clear();
+  bool IsInitialized() const;
+
+  int ByteSize() const;
+  bool MergePartialFromCodedStream(
+      ::google::protobuf::io::CodedInputStream* input);
+  void SerializeWithCachedSizes(
+      ::google::protobuf::io::CodedOutputStream* output) const;
+  ::google::protobuf::uint8* SerializeWithCachedSizesToArray(::google::protobuf::uint8* output) const;
+  int GetCachedSize() const { return _cached_size_; }
+  private:
+  void SharedCtor();
+  void SharedDtor();
+  void SetCachedSize(int size) const;
+  public:
+
+  ::google::protobuf::Metadata GetMetadata() const;
+
+  // nested types ----------------------------------------------------
+
+  // accessors -------------------------------------------------------
+
+  // optional .exec.user.RequestStatus status = 1;
+  inline bool has_status() const;
+  inline void clear_status();
+  static const int kStatusFieldNumber = 1;
+  inline ::exec::user::RequestStatus status() const;
+  inline void set_status(::exec::user::RequestStatus value);
+
+  // repeated .exec.user.SchemaMetadata schemas = 2;
+  inline int schemas_size() const;
+  inline void clear_schemas();
+  static const int kSchemasFieldNumber = 2;
+  inline const ::exec::user::SchemaMetadata& schemas(int index) const;
+  inline ::exec::user::SchemaMetadata* mutable_schemas(int index);
+  inline ::exec::user::SchemaMetadata* add_schemas();
+  inline const ::google::protobuf::RepeatedPtrField< ::exec::user::SchemaMetadata >&
+      schemas() const;
+  inline ::google::protobuf::RepeatedPtrField< ::exec::user::SchemaMetadata >*
+      mutable_schemas();
+
+  // optional .exec.shared.DrillPBError error = 3;
+  inline bool has_error() const;
+  inline void clear_error();
+  static const int kErrorFieldNumber = 3;
+  inline const ::exec::shared::DrillPBError& error() const;
+  inline ::exec::shared::DrillPBError* mutable_error();
+  inline ::exec::shared::DrillPBError* release_error();
+  inline void set_allocated_error(::exec::shared::DrillPBError* error);
+
+  // @@protoc_insertion_point(class_scope:exec.user.GetSchemasResp)
+ private:
+  inline void set_has_status();
+  inline void clear_has_status();
+  inline void set_has_error();
+  inline void clear_has_error();
+
+  ::google::protobuf::UnknownFieldSet _unknown_fields_;
+
+  ::google::protobuf::RepeatedPtrField< ::exec::user::SchemaMetadata > schemas_;
+  ::exec::shared::DrillPBError* error_;
+  int status_;
+
+  mutable int _cached_size_;
+  ::google::protobuf::uint32 _has_bits_[(3 + 31) / 32];
+
+  friend void  protobuf_AddDesc_User_2eproto();
+  friend void protobuf_AssignDesc_User_2eproto();
+  friend void protobuf_ShutdownFile_User_2eproto();
+
+  void InitAsDefaultInstance();
+  static GetSchemasResp* default_instance_;
+};
+// -------------------------------------------------------------------
+
+class GetTablesReq : public ::google::protobuf::Message {
+ public:
+  GetTablesReq();
+  virtual ~GetTablesReq();
+
+  GetTablesReq(const GetTablesReq& from);
+
+  inline GetTablesReq& operator=(const GetTablesReq& from) {
+    CopyFrom(from);
+    return *this;
+  }
+
+  inline const ::google::protobuf::UnknownFieldSet& unknown_fields() const {
+    return _unknown_fields_;
+  }
+
+  inline ::google::protobuf::UnknownFieldSet* mutable_unknown_fields() {
+    return &_unknown_fields_;
+  }
+
+  static const ::google::protobuf::Descriptor* descriptor();
+  static const GetTablesReq& default_instance();
+
+  void Swap(GetTablesReq* other);
+
+  // implements Message ----------------------------------------------
+
+  GetTablesReq* New() const;
+  void CopyFrom(const ::google::protobuf::Message& from);
+  void MergeFrom(const ::google::protobuf::Message& from);
+  void CopyFrom(const GetTablesReq& from);
+  void MergeFrom(const GetTablesReq& from);
+  void Clear();
+  bool IsInitialized() const;
+
+  int ByteSize() const;
+  bool MergePartialFromCodedStream(
+      ::google::protobuf::io::CodedInputStream* input);
+  void SerializeWithCachedSizes(
+      ::google::protobuf::io::CodedOutputStream* output) const;
+  ::google::protobuf::uint8* SerializeWithCachedSizesToArray(::google::protobuf::uint8* output) const;
+  int GetCachedSize() const { return _cached_size_; }
+  private:
+  void SharedCtor();
+  void SharedDtor();
+  void SetCachedSize(int size) const;
+  public:
+
+  ::google::protobuf::Metadata GetMetadata() const;
+
+  // nested types ----------------------------------------------------
+
+  // accessors -------------------------------------------------------
+
+  // optional .exec.user.LikeFilter catalog_name_filter = 1;
+  inline bool has_catalog_name_filter() const;
+  inline void clear_catalog_name_filter();
+  static const int kCatalogNameFilterFieldNumber = 1;
+  inline const ::exec::user::LikeFilter& catalog_name_filter() const;
+  inline ::exec::user::LikeFilter* mutable_catalog_name_filter();
+  inline ::exec::user::LikeFilter* release_catalog_name_filter();
+  inline void set_allocated_catalog_name_filter(::exec::user::LikeFilter* catalog_name_filter);
+
+  // optional .exec.user.LikeFilter schema_name_filter = 2;
+  inline bool has_schema_name_filter() const;
+  inline void clear_schema_name_filter();
+  static const int kSchemaNameFilterFieldNumber = 2;
+  inline const ::exec::user::LikeFilter& schema_name_filter() const;
+  inline ::exec::user::LikeFilter* mutable_schema_name_filter();
+  inline ::exec::user::LikeFilter* release_schema_name_filter();
+  inline void set_allocated_schema_name_filter(::exec::user::LikeFilter* schema_name_filter);
+
+  // optional .exec.user.LikeFilter table_name_filter = 3;
+  inline bool has_table_name_filter() const;
+  inline void clear_table_name_filter();
+  static const int kTableNameFilterFieldNumber = 3;
+  inline const ::exec::user::LikeFilter& table_name_filter() const;
+  inline ::exec::user::LikeFilter* mutable_table_name_filter();
+  inline ::exec::user::LikeFilter* release_table_name_filter();
+  inline void set_allocated_table_name_filter(::exec::user::LikeFilter* table_name_filter);
+
+  // repeated string table_type_filter = 4;
+  inline int table_type_filter_size() const;
+  inline void clear_table_type_filter();
+  static const int kTableTypeFilterFieldNumber = 4;
+  inline const ::std::string& table_type_filter(int index) const;
+  inline ::std::string* mutable_table_type_filter(int index);
+  inline void set_table_type_filter(int index, const ::std::string& value);
+  inline void set_table_type_filter(int index, const char* value);
+  inline void set_table_type_filter(int index, const char* value, size_t size);
+  inline ::std::string* add_table_type_filter();
+  inline void add_table_type_filter(const ::std::string& value);
+  inline void add_table_type_filter(const char* value);
+  inline void add_table_type_filter(const char* value, size_t size);
+  inline const ::google::protobuf::RepeatedPtrField< ::std::string>& table_type_filter() const;
+  inline ::google::protobuf::RepeatedPtrField< ::std::string>* mutable_table_type_filter();
+
+  // @@protoc_insertion_point(class_scope:exec.user.GetTablesReq)
+ private:
+  inline void set_has_catalog_name_filter();
+  inline void clear_has_catalog_name_filter();
+  inline void set_has_schema_name_filter();
+  inline void clear_has_schema_name_filter();
+  inline void set_has_table_name_filter();
+  inline void clear_has_table_name_filter();
+
+  ::google::protobuf::UnknownFieldSet _unknown_fields_;
+
+  ::exec::user::LikeFilter* catalog_name_filter_;
+  ::exec::user::LikeFilter* schema_name_filter_;
+  ::exec::user::LikeFilter* table_name_filter_;
+  ::google::protobuf::RepeatedPtrField< ::std::string> table_type_filter_;
+
+  mutable int _cached_size_;
+  ::google::protobuf::uint32 _has_bits_[(4 + 31) / 32];
+
+  friend void  protobuf_AddDesc_User_2eproto();
+  friend void protobuf_AssignDesc_User_2eproto();
+  friend void protobuf_ShutdownFile_User_2eproto();
+
+  void InitAsDefaultInstance();
+  static GetTablesReq* default_instance_;
+};
+// -------------------------------------------------------------------
+
+class TableMetadata : public ::google::protobuf::Message {
+ public:
+  TableMetadata();
+  virtual ~TableMetadata();
+
+  TableMetadata(const TableMetadata& from);
+
+  inline TableMetadata& operator=(const TableMetadata& from) {
+    CopyFrom(from);
+    return *this;
+  }
+
+  inline const ::google::protobuf::UnknownFieldSet& unknown_fields() const {
+    return _unknown_fields_;
+  }
+
+  inline ::google::protobuf::UnknownFieldSet* mutable_unknown_fields() {
+    return &_unknown_fields_;
+  }
+
+  static const ::google::protobuf::Descriptor* descriptor();
+  static const TableMetadata& default_instance();
+
+  void Swap(TableMetadata* other);
+
+  // implements Message ----------------------------------------------
+
+  TableMetadata* New() const;
+  void CopyFrom(const ::google::protobuf::Message& from);
+  void MergeFrom(const ::google::protobuf::Message& from);
+  void CopyFrom(const TableMetadata& from);
+  void MergeFrom(const TableMetadata& from);
+  void Clear();
+  bool IsInitialized() const;
+
+  int ByteSize() const;
+  bool MergePartialFromCodedStream(
+      ::google::protobuf::io::CodedInputStream* input);
+  void SerializeWithCachedSizes(
+      ::google::protobuf::io::CodedOutputStream* output) const;
+  ::google::protobuf::uint8* SerializeWithCachedSizesToArray(::google::protobuf::uint8* output) const;
+  int GetCachedSize() const { return _cached_size_; }
+  private:
+  void SharedCtor();
+  void SharedDtor();
+  void SetCachedSize(int size) const;
+  public:
+
+  ::google::protobuf::Metadata GetMetadata() const;
+
+  // nested types ----------------------------------------------------
+
+  // accessors -------------------------------------------------------
+
+  // optional string catalog_name = 1;
+  inline bool has_catalog_name() const;
+  inline void clear_catalog_name();
+  static const int kCatalogNameFieldNumber = 1;
+  inline const ::std::string& catalog_name() const;
+  inline void set_catalog_name(const ::std::string& value);
+  inline void set_catalog_name(const char* value);
+  inline void set_catalog_name(const char* value, size_t size);
+  inline ::std::string* mutable_catalog_name();
+  inline ::std::string* release_catalog_name();
+  inline void set_allocated_catalog_name(::std::string* catalog_name);
+
+  // optional string schema_name = 2;
+  inline bool has_schema_name() const;
+  inline void clear_schema_name();
+  static const int kSchemaNameFieldNumber = 2;
+  inline const ::std::string& schema_name() const;
+  inline void set_schema_name(const ::std::string& value);
+  inline void set_schema_name(const char* value);
+  inline void set_schema_name(const char* value, size_t size);
+  inline ::std::string* mutable_schema_name();
+  inline ::std::string* release_schema_name();
+  inline void set_allocated_schema_name(::std::string* schema_name);
+
+  // optional string table_name = 3;
+  inline bool has_table_name() const;
+  inline void clear_table_name();
+  static const int kTableNameFieldNumber = 3;
+  inline const ::std::string& table_name() const;
+  inline void set_table_name(const ::std::string& value);
+  inline void set_table_name(const char* value);
+  inline void set_table_name(const char* value, size_t size);
+  inline ::std::string* mutable_table_name();
+  inline ::std::string* release_table_name();
+  inline void set_allocated_table_name(::std::string* table_name);
+
+  // optional string type = 4;
+  inline bool has_type() const;
+  inline void clear_type();
+  static const int kTypeFieldNumber = 4;
+  inline const ::std::string& type() const;
+  inline void set_type(const ::std::string& value);
+  inline void set_type(const char* value);
+  inline void set_type(const char* value, size_t size);
+  inline ::std::string* mutable_type();
+  inline ::std::string* release_type();
+  inline void set_allocated_type(::std::string* type);
+
+  // @@protoc_insertion_point(class_scope:exec.user.TableMetadata)
+ private:
+  inline void set_has_catalog_name();
+  inline void clear_has_catalog_name();
+  inline void set_has_schema_name();
+  inline void clear_has_schema_name();
+  inline void set_has_table_name();
+  inline void clear_has_table_name();
+  inline void set_has_type();
+  inline void clear_has_type();
+
+  ::google::protobuf::UnknownFieldSet _unknown_fields_;
+
+  ::std::string* catalog_name_;
+  ::std::string* schema_name_;
+  ::std::string* table_name_;
+  ::std::string* type_;
+
+  mutable int _cached_size_;
+  ::google::protobuf::uint32 _has_bits_[(4 + 31) / 32];
+
+  friend void  protobuf_AddDesc_User_2eproto();
+  friend void protobuf_AssignDesc_User_2eproto();
+  friend void protobuf_ShutdownFile_User_2eproto();
+
+  void InitAsDefaultInstance();
+  static TableMetadata* default_instance_;
+};
+// -------------------------------------------------------------------
+
+class GetTablesResp : public ::google::protobuf::Message {
+ public:
+  GetTablesResp();
+  virtual ~GetTablesResp();
+
+  GetTablesResp(const GetTablesResp& from);
+
+  inline GetTablesResp& operator=(const GetTablesResp& from) {
+    CopyFrom(from);
+    return *this;
+  }
+
+  inline const ::google::protobuf::UnknownFieldSet& unknown_fields() const {
+    return _unknown_fields_;
+  }
+
+  inline ::google::protobuf::UnknownFieldSet* mutable_unknown_fields() {
+    return &_unknown_fields_;
+  }
+
+  static const ::google::protobuf::Descriptor* descriptor();
+  static const GetTablesResp& default_instance();
+
+  void Swap(GetTablesResp* other);
+
+  // implements Message ----------------------------------------------
+
+  GetTablesResp* New() const;
+  void CopyFrom(const ::google::protobuf::Message& from);
+  void MergeFrom(const ::google::protobuf::Message& from);
+  void CopyFrom(const GetTablesResp& from);
+  void MergeFrom(const GetTablesResp& from);
+  void Clear();
+  bool IsInitialized() const;
+
+  int ByteSize() const;
+  bool MergePartialFromCodedStream(
+      ::google::protobuf::io::CodedInputStream* input);
+  void SerializeWithCachedSizes(
+      ::google::protobuf::io::CodedOutputStream* output) const;
+  ::google::protobuf::uint8* SerializeWithCachedSizesToArray(::google::protobuf::uint8* output) const;
+  int GetCachedSize() const { return _cached_size_; }
+  private:
+  void SharedCtor();
+  void SharedDtor();
+  void SetCachedSize(int size) const;
+  public:
+
+  ::google::protobuf::Metadata GetMetadata() const;
+
+  // nested types ----------------------------------------------------
+
+  // accessors -------------------------------------------------------
+
+  // optional .exec.user.RequestStatus status = 1;
+  inline bool has_status() const;
+  inline void clear_status();
+  static const int kStatusFieldNumber = 1;
+  inline ::exec::user::RequestStatus status() const;
+  inline void set_status(::exec::user::RequestStatus value);
+
+  // repeated .exec.user.TableMetadata tables = 2;
+  inline int tables_size() const;
+  inline void clear_tables();
+  static const int kTablesFieldNumber = 2;
+  inline const ::exec::user::TableMetadata& tables(int index) const;
+  inline ::exec::user::TableMetadata* mutable_tables(int index);
+  inline ::exec::user::TableMetadata* add_tables();
+  inline const ::google::protobuf::RepeatedPtrField< ::exec::user::TableMetadata >&
+      tables() const;
+  inline ::google::protobuf::RepeatedPtrField< ::exec::user::TableMetadata >*
+      mutable_tables();
+
+  // optional .exec.shared.DrillPBError error = 3;
+  inline bool has_error() const;
+  inline void clear_error();
+  static const int kErrorFieldNumber = 3;
+  inline const ::exec::shared::DrillPBError& error() const;
+  inline ::exec::shared::DrillPBError* mutable_error();
+  inline ::exec::shared::DrillPBError* release_error();
+  inline void set_allocated_error(::exec::shared::DrillPBError* error);
+
+  // @@protoc_insertion_point(class_scope:exec.user.GetTablesResp)
+ private:
+  inline void set_has_status();
+  inline void clear_has_status();
+  inline void set_has_error();
+  inline void clear_has_error();
+
+  ::google::protobuf::UnknownFieldSet _unknown_fields_;
+
+  ::google::protobuf::RepeatedPtrField< ::exec::user::TableMetadata > tables_;
+  ::exec::shared::DrillPBError* error_;
+  int status_;
+
+  mutable int _cached_size_;
+  ::google::protobuf::uint32 _has_bits_[(3 + 31) / 32];
+
+  friend void  protobuf_AddDesc_User_2eproto();
+  friend void protobuf_AssignDesc_User_2eproto();
+  friend void protobuf_ShutdownFile_User_2eproto();
+
+  void InitAsDefaultInstance();
+  static GetTablesResp* default_instance_;
+};
+// -------------------------------------------------------------------
+
+class GetColumnsReq : public ::google::protobuf::Message {
+ public:
+  GetColumnsReq();
+  virtual ~GetColumnsReq();
+
+  GetColumnsReq(const GetColumnsReq& from);
+
+  inline GetColumnsReq& operator=(const GetColumnsReq& from) {
+    CopyFrom(from);
+    return *this;
+  }
+
+  inline const ::google::protobuf::UnknownFieldSet& unknown_fields() const {
+    return _unknown_fields_;
+  }
+
+  inline ::google::protobuf::UnknownFieldSet* mutable_unknown_fields() {
+    return &_unknown_fields_;
+  }
+
+  static const ::google::protobuf::Descriptor* descriptor();
+  static const GetColumnsReq& default_instance();
+
+  void Swap(GetColumnsReq* other);
+
+  // implements Message ----------------------------------------------
+
+  GetColumnsReq* New() const;
+  void CopyFrom(const ::google::protobuf::Message& from);
+  void MergeFrom(const ::google::protobuf::Message& from);
+  void CopyFrom(const GetColumnsReq& from);
+  void MergeFrom(const GetColumnsReq& from);
+  void Clear();
+  bool IsInitialized() const;
+
+  int ByteSize() const;
+  bool MergePartialFromCodedStream(
+      ::google::protobuf::io::CodedInputStream* input);
+  void SerializeWithCachedSizes(
+      ::google::protobuf::io::CodedOutputStream* output) const;
+  ::google::protobuf::uint8* SerializeWithCachedSizesToArray(::google::protobuf::uint8* output) const;
+  int GetCachedSize() const { return _cached_size_; }
+  private:
+  void SharedCtor();
+  void SharedDtor();
+  void SetCachedSize(int size) const;
+  public:
+
+  ::google::protobuf::Metadata GetMetadata() const;
+
+  // nested types ----------------------------------------------------
+
+  // accessors -------------------------------------------------------
+
+  // optional .exec.user.LikeFilter catalog_name_filter = 1;
+  inline bool has_catalog_name_filter() const;
+  inline void clear_catalog_name_filter();
+  static const int kCatalogNameFilterFieldNumber = 1;
+  inline const ::exec::user::LikeFilter& catalog_name_filter() const;
+  inline ::exec::user::LikeFilter* mutable_catalog_name_filter();
+  inline ::exec::user::LikeFilter* release_catalog_name_filter();
+  inline void set_allocated_catalog_name_filter(::exec::user::LikeFilter* catalog_name_filter);
+
+  // optional .exec.user.LikeFilter schema_name_filter = 2;
+  inline bool has_schema_name_filter() const;
+  inline void clear_schema_name_filter();
+  static const int kSchemaNameFilterFieldNumber = 2;
+  inline const ::exec::user::LikeFilter& schema_name_filter() const;
+  inline ::exec::user::LikeFilter* mutable_schema_name_filter();
+  inline ::exec::user::LikeFilter* release_schema_name_filter();
+  inline void set_allocated_schema_name_filter(::exec::user::LikeFilter* schema_name_filter);
+
+  // optional .exec.user.LikeFilter table_name_filter = 3;
+  inline bool has_table_name_filter() const;
+  inline void clear_table_name_filter();
+  static const int kTableNameFilterFieldNumber = 3;
+  inline const ::exec::user::LikeFilter& table_name_filter() const;
+  inline ::exec::user::LikeFilter* mutable_table_name_filter();
+  inline ::exec::user::LikeFilter* release_table_name_filter();
+  inline void set_allocated_table_name_filter(::exec::user::LikeFilter* table_name_filter);
+
+  // optional .exec.user.LikeFilter column_name_filter = 4;
+  inline bool has_column_name_filter() const;
+  inline void clear_column_name_filter();
+  static const int kColumnNameFilterFieldNumber = 4;
+  inline const ::exec::user::LikeFilter& column_name_filter() const;
+  inline ::exec::user::LikeFilter* mutable_column_name_filter();
+  inline ::exec::user::LikeFilter* release_column_name_filter();
+  inline void set_allocated_column_name_filter(::exec::user::LikeFilter* column_name_filter);
+
+  // @@protoc_insertion_point(class_scope:exec.user.GetColumnsReq)
+ private:
+  inline void set_has_catalog_name_filter();
+  inline void clear_has_catalog_name_filter();
+  inline void set_has_schema_name_filter();
+  inline void clear_has_schema_name_filter();
+  inline void set_has_table_name_filter();
+  inline void clear_has_table_name_filter();
+  inline void set_has_column_name_filter();
+  inline void clear_has_column_name_filter();
+
+  ::google::protobuf::UnknownFieldSet _unknown_fields_;
+
+  ::exec::user::LikeFilter* catalog_name_filter_;
+  ::exec::user::LikeFilter* schema_name_filter_;
+  ::exec::user::LikeFilter* table_name_filter_;
+  ::exec::user::LikeFilter* column_name_filter_;
+
+  mutable int _cached_size_;
+  ::google::protobuf::uint32 _has_bits_[(4 + 31) / 32];
+
+  friend void  protobuf_AddDesc_User_2eproto();
+  friend void protobuf_AssignDesc_User_2eproto();
+  friend void protobuf_ShutdownFile_User_2eproto();
+
+  void InitAsDefaultInstance();
+  static GetColumnsReq* default_instance_;
+};
+// -------------------------------------------------------------------
+
+class ColumnMetadata : public ::google::protobuf::Message {
+ public:
+  ColumnMetadata();
+  virtual ~ColumnMetadata();
+
+  ColumnMetadata(const ColumnMetadata& from);
+
+  inline ColumnMetadata& operator=(const ColumnMetadata& from) {
+    CopyFrom(from);
+    return *this;
+  }
+
+  inline const ::google::protobuf::UnknownFieldSet& unknown_fields() const {
+    return _unknown_fields_;
+  }
+
+  inline ::google::protobuf::UnknownFieldSet* mutable_unknown_fields() {
+    return &_unknown_fields_;
+  }
+
+  static const ::google::protobuf::Descriptor* descriptor();
+  static const ColumnMetadata& default_instance();
+
+  void Swap(ColumnMetadata* other);
+
+  // implements Message ----------------------------------------------
+
+  ColumnMetadata* New() const;
+  void CopyFrom(const ::google::protobuf::Message& from);
+  void MergeFrom(const ::google::protobuf::Message& from);
+  void CopyFrom(const ColumnMetadata& from);
+  void MergeFrom(const ColumnMetadata& from);
+  void Clear();
+  bool IsInitialized() const;
+
+  int ByteSize() const;
+  bool MergePartialFromCodedStream(
+      ::google::protobuf::io::CodedInputStream* input);
+  void SerializeWithCachedSizes(
+      ::google::protobuf::io::CodedOutputStream* output) const;
+  ::google::protobuf::uint8* SerializeWithCachedSizesToArray(::google::protobuf::uint8* output) const;
+  int GetCachedSize() const { return _cached_size_; }
+  private:
+  void SharedCtor();
+  void SharedDtor();
+  void SetCachedSize(int size) const;
+  public:
+
+  ::google::protobuf::Metadata GetMetadata() const;
+
+  // nested types ----------------------------------------------------
+
+  // accessors -------------------------------------------------------
+
+  // optional string catalog_name = 1;
+  inline bool has_catalog_name() const;
+  inline void clear_catalog_name();
+  static const int kCatalogNameFieldNumber = 1;
+  inline const ::std::string& catalog_name() const;
+  inline void set_catalog_name(const ::std::string& value);
+  inline void set_catalog_name(const char* value);
+  inline void set_catalog_name(const char* value, size_t size);
+  inline ::std::string* mutable_catalog_name();
+  inline ::std::string* release_catalog_name();
+  inline void set_allocated_catalog_name(::std::string* catalog_name);
+
+  // optional string schema_name = 2;
+  inline bool has_schema_name() const;
+  inline void clear_schema_name();
+  static const int kSchemaNameFieldNumber = 2;
+  inline const ::std::string& schema_name() const;
+  inline void set_schema_name(const ::std::string& value);
+  inline void set_schema_name(const char* value);
+  inline void set_schema_name(const char* value, size_t size);
+  inline ::std::string* mutable_schema_name();
+  inline ::std::string* release_schema_name();
+  inline void set_allocated_schema_name(::std::string* schema_name);
+
+  // optional string table_name = 3;
+  inline bool has_table_name() const;
+  inline void clear_table_name();
+  static const int kTableNameFieldNumber = 3;
+  inline const ::std::string& table_name() const;
+  inline void set_table_name(const ::std::string& value);
+  inline void set_table_name(const char* value);
+  inline void set_table_name(const char* value, size_t size);
+  inline ::std::string* mutable_table_name();
+  inline ::std::string* release_table_name();
+  inline void set_allocated_table_name(::std::string* table_name);
+
+  // optional string column_name = 4;
+  inline bool has_column_name() const;
+  inline void clear_column_name();
+  static const int kColumnNameFieldNumber = 4;
+  inline const ::std::string& column_name() const;
+  inline void set_column_name(const ::std::string& value);
+  inline void set_column_name(const char* value);
+  inline void set_column_name(const char* value, size_t size);
+  inline ::std::string* mutable_column_name();
+  inline ::std::string* release_column_name();
+  inline void set_allocated_column_name(::std::string* column_name);
+
+  // optional int32 ordinal_position = 5;
+  inline bool has_ordinal_position() const;
+  inline void clear_ordinal_position();
+  static const int kOrdinalPositionFieldNumber = 5;
+  inline ::google::protobuf::int32 ordinal_position() const;
+  inline void set_ordinal_position(::google::protobuf::int32 value);
+
+  // optional string default_value = 6;
+  inline bool has_default_value() const;
+  inline void clear_default_value();
+  static const int kDefaultValueFieldNumber = 6;
+  inline const ::std::string& default_value() const;
+  inline void set_default_value(const ::std::string& value);
+  inline void set_default_value(const char* value);
+  inline void set_default_value(const char* value, size_t size);
+  inline ::std::string* mutable_default_value();
+  inline ::std::string* release_default_value();
+  inline void set_allocated_default_value(::std::string* default_value);
+
+  // optional bool is_nullable = 7;
+  inline bool has_is_nullable() const;
+  inline void clear_is_nullable();
+  static const int kIsNullableFieldNumber = 7;
+  inline bool is_nullable() const;
+  inline void set_is_nullable(bool value);
+
+  // optional string data_type = 8;
+  inline bool has_data_type() const;
+  inline void clear_data_type();
+  static const int kDataTypeFieldNumber = 8;
+  inline const ::std::string& data_type() const;
+  inline void set_data_type(const ::std::string& value);
+  inline void set_data_type(const char* value);
+  inline void set_data_type(const char* value, size_t size);
+  inline ::std::string* mutable_data_type();
+  inline ::std::string* release_data_type();
+  inline void set_allocated_data_type(::std::string* data_type);
+
+  // optional int32 char_max_length = 9;
+  inline bool has_char_max_length() const;
+  inline void clear_char_max_length();
+  static const int kCharMaxLengthFieldNumber = 9;
+  inline ::google::protobuf::int32 char_max_length() const;
+  inline void set_char_max_length(::google::protobuf::int32 value);
+
+  // optional int32 char_octet_length = 10;
+  inline bool has_char_octet_length() const;
+  inline void clear_char_octet_length();
+  static const int kCharOctetLengthFieldNumber = 10;
+  inline ::google::protobuf::int32 char_octet_length() const;
+  inline void set_char_octet_length(::google::protobuf::int32 value);
+
+  // optional int32 numeric_precision = 11;
+  inline bool has_numeric_precision() const;
+  inline void clear_numeric_precision();
+  static const int kNumericPrecisionFieldNumber = 11;
+  inline ::google::protobuf::int32 numeric_precision() const;
+  inline void set_numeric_precision(::google::protobuf::int32 value);
+
+  // optional int32 numeric_precision_radix = 12;
+  inline bool has_numeric_precision_radix() const;
+  inline void clear_numeric_precision_radix();
+  static const int kNumericPrecisionRadixFieldNumber = 12;
+  inline ::google::protobuf::int32 numeric_precision_radix() const;
+  inline void set_numeric_precision_radix(::google::protobuf::int32 value);
+
+  // optional int32 numeric_scale = 13;
+  inline bool has_numeric_scale() const;
+  inline void clear_numeric_scale();
+  static const int kNumericScaleFieldNumber = 13;
+  inline ::google::protobuf::int32 numeric_scale() const;
+  inline void set_numeric_scale(::google::protobuf::int32 value);
+
+  // optional int32 date_time_precision = 14;
+  inline bool has_date_time_precision() const;
+  inline void clear_date_time_precision();
+  static const int kDateTimePrecisionFieldNumber = 14;
+  inline ::google::protobuf::int32 date_time_precision() const;
+  inline void set_date_time_precision(::google::protobuf::int32 value);
+
+  // optional string interval_type = 15;
+  inline bool has_interval_type() const;
+  inline void clear_interval_type();
+  static const int kIntervalTypeFieldNumber = 15;
+  inline const ::std::string& interval_type() const;
+  inline void set_interval_type(const ::std::string& value);
+  inline void set_interval_type(const char* value);
+  inline void set_interval_type(const char* value, size_t size);
+  inline ::std::string* mutable_interval_type();
+  inline ::std::string* release_interval_type();
+  inline void set_allocated_interval_type(::std::string* interval_type);
+
+  // optional int32 interval_precision = 16;
+  inline bool has_interval_precision() const;
+  inline void clear_interval_precision();
+  static const int kIntervalPrecisionFieldNumber = 16;
+  inline ::google::protobuf::int32 interval_precision() const;
+  inline void set_interval_precision(::google::protobuf::int32 value);
+
+  // optional int32 column_size = 17;
+  inline bool has_column_size() const;
+  inline void clear_column_size();
+  static const int kColumnSizeFieldNumber = 17;
+  inline ::google::protobuf::int32 column_size() const;
+  inline void set_column_size(::google::protobuf::int32 value);
+
+  // @@protoc_insertion_point(class_scope:exec.user.ColumnMetadata)
+ private:
+  inline void set_has_catalog_name();
+  inline void clear_has_catalog_name();
+  inline void set_has_schema_name();
+  inline void clear_has_schema_name();
+  inline void set_has_table_name();
+  inline void clear_has_table_name();
+  inline void set_has_column_name();
+  inline void clear_has_column_name();
+  inline void set_has_ordinal_position();
+  inline void clear_has_ordinal_position();
+  inline void set_has_default_value();
+  inline void clear_has_default_value();
+  inline void set_has_is_nullable();
+  inline void clear_has_is_nullable();
+  inline void set_has_data_type();
+  inline void clear_has_data_type();
+  inline void set_has_char_max_length();
+  inline void clear_has_char_max_length();
+  inline void set_has_char_octet_length();
+  inline void clear_has_char_octet_length();
+  inline void set_has_numeric_precision();
+  inline void clear_has_numeric_precision();
+  inline void set_has_numeric_precision_radix();
+  inline void clear_has_numeric_precision_radix();
+  inline void set_has_numeric_scale();
+  inline void clear_has_numeric_scale();
+  inline void set_has_date_time_precision();
+  inline void clear_has_date_time_precision();
+  inline void set_has_interval_type();
+  inline void clear_has_interval_type();
+  inline void set_has_interval_precision();
+  inline void clear_has_interval_precision();
+  inline void set_has_column_size();
+  inline void clear_has_column_size();
+
+  ::google::protobuf::UnknownFieldSet _unknown_fields_;
+
+  ::std::string* catalog_name_;
+  ::std::string* schema_name_;
+  ::std::string* table_name_;
+  ::std::string* column_name_;
+  ::std::string* default_value_;
+  ::google::protobuf::int32 ordinal_position_;
+  bool is_nullable_;
+  ::std::string* data_type_;
+  ::google::protobuf::int32 char_max_length_;
+  ::google::protobuf::int32 char_octet_length_;
+  ::google::protobuf::int32 numeric_precision_;
+  ::google::protobuf::int32 numeric_precision_radix_;
+  ::google::protobuf::int32 numeric_scale_;
+  ::google::protobuf::int32 date_time_precision_;
+  ::std::string* interval_type_;
+  ::google::protobuf::int32 interval_precision_;
+  ::google::protobuf::int32 column_size_;
+
+  mutable int _cached_size_;
+  ::google::protobuf::uint32 _has_bits_[(17 + 31) / 32];
+
+  friend void  protobuf_AddDesc_User_2eproto();
+  friend void protobuf_AssignDesc_User_2eproto();
+  friend void protobuf_ShutdownFile_User_2eproto();
+
+  void InitAsDefaultInstance();
+  static ColumnMetadata* default_instance_;
+};
+// -------------------------------------------------------------------
+
+class GetColumnsResp : public ::google::protobuf::Message {
+ public:
+  GetColumnsResp();
+  virtual ~GetColumnsResp();
+
+  GetColumnsResp(const GetColumnsResp& from);
+
+  inline GetColumnsResp& operator=(const GetColumnsResp& from) {
+    CopyFrom(from);
+    return *this;
+  }
+
+  inline const ::google::protobuf::UnknownFieldSet& unknown_fields() const {
+    return _unknown_fields_;
+  }
+
+  inline ::google::protobuf::UnknownFieldSet* mutable_unknown_fields() {
+    return &_unknown_fields_;
+  }
+
+  static const ::google::protobuf::Descriptor* descriptor();
+  static const GetColumnsResp& default_instance();
+
+  void Swap(GetColumnsResp* other);
+
+  // implements Message ----------------------------------------------
+
+  GetColumnsResp* New() const;
+  void CopyFrom(const ::google::protobuf::Message& from);
+  void MergeFrom(const ::google::protobuf::Message& from);
+  void CopyFrom(const GetColumnsResp& from);
+  void MergeFrom(const GetColumnsResp& from);
+  void Clear();
+  bool IsInitialized() const;
+
+  int ByteSize() const;
+  bool MergePartialFromCodedStream(
+      ::google::protobuf::io::CodedInputStream* input);
+  void SerializeWithCachedSizes(
+      ::google::protobuf::io::CodedOutputStream* output) const;
+  ::google::protobuf::uint8* SerializeWithCachedSizesToArray(::google::protobuf::uint8* output) const;
+  int GetCachedSize() const { return _cached_size_; }
+  private:
+  void SharedCtor();
+  void SharedDtor();
+  void SetCachedSize(int size) const;
+  public:
+
+  ::google::protobuf::Metadata GetMetadata() const;
+
+  // nested types ----------------------------------------------------
+
+  // accessors -------------------------------------------------------
+
+  // optional .exec.user.RequestStatus status = 1;
+  inline bool has_status() const;
+  inline void clear_status();
+  static const int kStatusFieldNumber = 1;
+  inline ::exec::user::RequestStatus status() const;
+  inline void set_status(::exec::user::RequestStatus value);
+
+  // repeated .exec.user.ColumnMetadata columns = 2;
+  inline int columns_size() const;
+  inline void clear_columns();
+  static const int kColumnsFieldNumber = 2;
+  inline const ::exec::user::ColumnMetadata& columns(int index) const;
+  inline ::exec::user::ColumnMetadata* mutable_columns(int index);
+  inline ::exec::user::ColumnMetadata* add_columns();
+  inline const ::google::protobuf::RepeatedPtrField< ::exec::user::ColumnMetadata >&
+      columns() const;
+  inline ::google::protobuf::RepeatedPtrField< ::exec::user::ColumnMetadata >*
+      mutable_columns();
+
+  // optional .exec.shared.DrillPBError error = 3;
+  inline bool has_error() const;
+  inline void clear_error();
+  static const int kErrorFieldNumber = 3;
+  inline const ::exec::shared::DrillPBError& error() const;
+  inline ::exec::shared::DrillPBError* mutable_error();
+  inline ::exec::shared::DrillPBError* release_error();
+  inline void set_allocated_error(::exec::shared::DrillPBError* error);
+
+  // @@protoc_insertion_point(class_scope:exec.user.GetColumnsResp)
+ private:
+  inline void set_has_status();
+  inline void clear_has_status();
+  inline void set_has_error();
+  inline void clear_has_error();
+
+  ::google::protobuf::UnknownFieldSet _unknown_fields_;
+
+  ::google::protobuf::RepeatedPtrField< ::exec::user::ColumnMetadata > columns_;
+  ::exec::shared::DrillPBError* error_;
+  int status_;
+
+  mutable int _cached_size_;
+  ::google::protobuf::uint32 _has_bits_[(3 + 31) / 32];
+
+  friend void  protobuf_AddDesc_User_2eproto();
+  friend void protobuf_AssignDesc_User_2eproto();
+  friend void protobuf_ShutdownFile_User_2eproto();
+
+  void InitAsDefaultInstance();
+  static GetColumnsResp* default_instance_;
+};
+// -------------------------------------------------------------------
+
+class CreatePreparedStatementReq : public ::google::protobuf::Message {
+ public:
+  CreatePreparedStatementReq();
+  virtual ~CreatePreparedStatementReq();
+
+  CreatePreparedStatementReq(const CreatePreparedStatementReq& from);
+
+  inline CreatePreparedStatementReq& operator=(const CreatePreparedStatementReq& from) {
+    CopyFrom(from);
+    return *this;
+  }
+
+  inline const ::google::protobuf::UnknownFieldSet& unknown_fields() const {
+    return _unknown_fields_;
+  }
+
+  inline ::google::protobuf::UnknownFieldSet* mutable_unknown_fields() {
+    return &_unknown_fields_;
+  }
+
+  static const ::google::protobuf::Descriptor* descriptor();
+  static const CreatePreparedStatementReq& default_instance();
+
+  void Swap(CreatePreparedStatementReq* other);
+
+  // implements Message ----------------------------------------------
+
+  CreatePreparedStatementReq* New() const;
+  void CopyFrom(const ::google::protobuf::Message& from);
+  void MergeFrom(const ::google::protobuf::Message& from);
+  void CopyFrom(const CreatePreparedStatementReq& from);
+  void MergeFrom(const CreatePreparedStatementReq& from);
+  void Clear();
+  bool IsInitialized() const;
+
+  int ByteSize() const;
+  bool MergePartialFromCodedStream(
+      ::google::protobuf::io::CodedInputStream* input);
+  void SerializeWithCachedSizes(
+      ::google::protobuf::io::CodedOutputStream* output) const;
+  ::google::protobuf::uint8* SerializeWithCachedSizesToArray(::google::protobuf::uint8* output) const;
+  int GetCachedSize() const { return _cached_size_; }
+  private:
+  void SharedCtor();
+  void SharedDtor();
+  void SetCachedSize(int size) const;
+  public:
+
+  ::google::protobuf::Metadata GetMetadata() const;
+
+  // nested types ----------------------------------------------------
+
+  // accessors -------------------------------------------------------
+
+  // optional string sql_query = 1;
+  inline bool has_sql_query() const;
+  inline void clear_sql_query();
+  static const int kSqlQueryFieldNumber = 1;
+  inline const ::std::string& sql_query() const;
+  inline void set_sql_query(const ::std::string& value);
+  inline void set_sql_query(const char* value);
+  inline void set_sql_query(const char* value, size_t size);
+  inline ::std::string* mutable_sql_query();
+  inline ::std::string* release_sql_query();
+  inline void set_allocated_sql_query(::std::string* sql_query);
+
+  // @@protoc_insertion_point(class_scope:exec.user.CreatePreparedStatementReq)
+ private:
+  inline void set_has_sql_query();
+  inline void clear_has_sql_query();
+
+  ::google::protobuf::UnknownFieldSet _unknown_fields_;
+
+  ::std::string* sql_query_;
+
+  mutable int _cached_size_;
+  ::google::protobuf::uint32 _has_bits_[(1 + 31) / 32];
+
+  friend void  protobuf_AddDesc_User_2eproto();
+  friend void protobuf_AssignDesc_User_2eproto();
+  friend void protobuf_ShutdownFile_User_2eproto();
+
+  void InitAsDefaultInstance();
+  static CreatePreparedStatementReq* default_instance_;
+};
+// -------------------------------------------------------------------
+
+class ResultColumnMetadata : public ::google::protobuf::Message {
+ public:
+  ResultColumnMetadata();
+  virtual ~ResultColumnMetadata();
+
+  ResultColumnMetadata(const ResultColumnMetadata& from);
+
+  inline ResultColumnMetadata& operator=(

<TRUNCATED>

[05/15] drill git commit: DRILL-1268: Add unit test to C++ native client

Posted by pa...@apache.org.
DRILL-1268: Add unit test to C++ native client

Add CppUnit unit test to the C++ native client


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

Branch: refs/heads/master
Commit: 3a35a4200e748ed557c55d6d13ac995cce28ab09
Parents: 2558803
Author: Laurent Goujon <la...@dremio.com>
Authored: Mon Aug 22 14:54:19 2016 -0700
Committer: Parth Chandra <pa...@apache.org>
Committed: Tue Nov 1 11:29:58 2016 -0700

----------------------------------------------------------------------
 contrib/native/client/CMakeLists.txt            | 44 ++++++++++++-
 .../client/cmakeModules/FindCppUnit.cmake       | 67 ++++++++++++++++++++
 .../client/cmakeModules/FindZookeeper.cmake     |  4 +-
 contrib/native/client/readme.win.txt            | 21 ++++--
 contrib/native/client/src/clientlib/utils.hpp   |  2 +-
 contrib/native/client/src/test/CMakeLists.txt   | 40 ++++++++++++
 contrib/native/client/src/test/UtilsTest.cpp    | 51 +++++++++++++++
 contrib/native/client/src/test/main.cpp         | 39 ++++++++++++
 8 files changed, 259 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/3a35a420/contrib/native/client/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/contrib/native/client/CMakeLists.txt b/contrib/native/client/CMakeLists.txt
index b53cb0a..e61eb9c 100644
--- a/contrib/native/client/CMakeLists.txt
+++ b/contrib/native/client/CMakeLists.txt
@@ -17,9 +17,47 @@
 #
 cmake_minimum_required(VERSION 2.6)
 
-project(drillclient)
+cmake_policy(SET CMP0043 NEW)
+cmake_policy(SET CMP0048 NEW)
+enable_testing()
+
+set (DRILL_ROOT ${CMAKE_SOURCE_DIR}/../../..)
+if (NOT DEFINED DRILL_VERSION)
+    message("Detecting Drill version:")
+    if (WIN32)
+        find_program(POWERSHELL Powershell)
+        file(TO_NATIVE_PATH "${DRILL_ROOT}/pom.xml" DRILL_POM_FILE)    
+        execute_process(
+            COMMAND ${POWERSHELL} "Select-Xml -Namespace @{'m'='http://maven.apache.org/POM/4.0.0'} -XPath '//m:project/m:version/text()' -Path ${DRILL_POM_FILE} | foreach {$_.Node.Value}"
+            WORKING_DIRECTORY ${CMAKE_SOURCE_DIR}
+            OUTPUT_VARIABLE DRILL_VERSION
+            OUTPUT_STRIP_TRAILING_WHITESPACE
+        )
+    else ()
+        find_program(MVN mvn)
+        execute_process(
+            COMMAND ${MVN} -q -f ${DRILL_ROOT}/pom.xml -Dexec.executable=echo -Dexec.args=\${project.version} --non-recursive org.codehaus.mojo:exec-maven-plugin:1.3.1:exec
+            WORKING_DIRECTORY ${CMAKE_SOURCE_DIR}
+            OUTPUT_VARIABLE DRILL_VERSION
+            OUTPUT_STRIP_TRAILING_WHITESPACE
+            )
+    endif ()
+    if ("${DRILL_VERSION}" STREQUAL "")
+        message(FATAL_ERROR "Cannot detect Drill version")
+    endif()
+endif()
+
+string(REGEX REPLACE "^([0-9]+\\.[0-9]+\\.[0-9]+).*" "\\1"
+       DRILL_VERSION
+       ${DRILL_VERSION}
+       )
+
+project(drillclient
+        VERSION ${DRILL_VERSION}
+        )
 
 message("Project Dir = ${PROJECT_SOURCE_DIR}")
+message("Project version = ${PROJECT_VERSION} ")
 message("Source Dir = ${CMAKE_SOURCE_DIR} ")
 
 cmake_policy(SET CMP0043 NEW)
@@ -76,7 +114,7 @@ find_package(Protobuf REQUIRED )
 include_directories(${PROTOBUF_INCLUDE_DIR})
 
 #Find Zookeeper
-find_package(Zookeeper  REQUIRED )
+find_package(Zookeeper REQUIRED )
 
 
 # Generated sources
@@ -105,6 +143,8 @@ add_subdirectory("${CMAKE_SOURCE_DIR}/src/clientlib/y2038")
 add_subdirectory("${CMAKE_SOURCE_DIR}/src/clientlib")
 include_directories(${CMAKE_SOURCE_DIR}/src/include ${Zookeeper_INCLUDE_DIRS})
 
+add_subdirectory("${CMAKE_SOURCE_DIR}/src/test")
+
 # add a DEBUG preprocessor macro
 set_property(
     DIRECTORY

http://git-wip-us.apache.org/repos/asf/drill/blob/3a35a420/contrib/native/client/cmakeModules/FindCppUnit.cmake
----------------------------------------------------------------------
diff --git a/contrib/native/client/cmakeModules/FindCppUnit.cmake b/contrib/native/client/cmakeModules/FindCppUnit.cmake
new file mode 100644
index 0000000..c7d6855
--- /dev/null
+++ b/contrib/native/client/cmakeModules/FindCppUnit.cmake
@@ -0,0 +1,67 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+# A simple cmake module to find CppUnit (inspired by
+# http://root.cern.ch/viewvc/trunk/cint/reflex/cmake/modules/FindCppUnit.cmake)
+
+#
+# - Find CppUnit
+# This module finds an installed CppUnit package.
+#
+# It sets the following variables:
+#  CPPUNIT_FOUND       - Set to false if CppUnit isn't found.
+#  CPPUNIT_INCLUDE_DIR - The CppUnit include directory.
+#  CPPUNIT_LIBRARY     - The CppUnit library to link against.
+
+if (MSVC)
+    if (${CMAKE_BUILD_TYPE} MATCHES "Debug")
+        set(CPPUNIT_BuildOutputDir "Debug")
+        set(CPPUNIT_LibName "cppunitd")
+    else()
+        set(CPPUNIT_BuildOutputDir "Release")
+        set(CPPUNIT_LibName "cppunit")
+    endif()
+    if ("${CPPUNIT_HOME}_" MATCHES  "^_$")
+        message(" ")
+        message("- Please set the cache variable CPPUNIT_HOME to point to the directory with the cppunit source.")
+        message("- CMAKE will look for cppunit include files in $CPPUNIT_HOME/include.")
+        message("- CMAKE will look for cppunit library files in $CPPUNIT_HOME/src/Debug or $CPPUNIT_HOME/src/Release.")
+    else()
+        file(TO_CMAKE_PATH ${CPPUNIT_HOME} CPPUNIT_HomePath)
+        set(CPPUNIT_LIB_PATHS ${CPPUNIT_HomePath}/src/cppunit/${CPPUNIT_BuildOutputDir})
+
+        find_path(CPPUNIT_INCLUDE_DIR cppunit/Test.h ${CPPUNIT_HomePath}/include)
+        find_library(CPPUNIT_LIBRARY NAMES ${CPPUNIT_LibName} PATHS ${CPPUNIT_LIB_PATHS})
+    endif()
+else()
+    set(CPPUNIT_LIB_PATHS /usr/local/lib /opt/local/lib)
+    find_path(CPPUNIT_INCLUDE_DIR cppunit/Test.h /usr/local/include /opt/local/include)
+    find_library(CPPUNIT_LIBRARY NAMES cppunit PATHS ${CPPUNIT_LIB_PATHS})
+endif()
+
+if (CPPUNIT_INCLUDE_DIR AND CPPUNIT_LIBRARY)
+    set(CPPUNIT_FOUND TRUE)
+else (CPPUNIT_INCLUDE_DIR AND CPPUNIT_LIBRARY)
+    set(CPPUNIT_FOUND FALSE)
+endif (CPPUNIT_INCLUDE_DIR AND CPPUNIT_LIBRARY)
+
+if (CPPUNIT_FOUND)
+    message(STATUS "Found CppUnit: ${CPPUNIT_LIBRARY}")
+else (CPPUNIT_FOUND)
+    message(WARNING "Could not find CppUnit: tests won't compile")
+endif (CPPUNIT_FOUND)

http://git-wip-us.apache.org/repos/asf/drill/blob/3a35a420/contrib/native/client/cmakeModules/FindZookeeper.cmake
----------------------------------------------------------------------
diff --git a/contrib/native/client/cmakeModules/FindZookeeper.cmake b/contrib/native/client/cmakeModules/FindZookeeper.cmake
index 151c05c..628bbcc 100644
--- a/contrib/native/client/cmakeModules/FindZookeeper.cmake
+++ b/contrib/native/client/cmakeModules/FindZookeeper.cmake
@@ -30,8 +30,10 @@
 if (MSVC)
     if(${CMAKE_BUILD_TYPE} MATCHES "Debug")
         set(ZK_BuildOutputDir "Debug")
+        set(ZK_LibName "zookeeper_d")
     else()
         set(ZK_BuildOutputDir "Release")
+        set(ZK_LibName "zookeeper")
     endif()
     if("${ZOOKEEPER_HOME}_" MATCHES  "^_$")
         message(" ")
@@ -45,7 +47,7 @@ if (MSVC)
         find_path(ZK_INCLUDE_DIR zookeeper.h ${Zookeeper_HomePath}/src/c/include)
         find_path(ZK_INCLUDE_DIR_GEN zookeeper.jute.h ${Zookeeper_HomePath}/src/c/generated)
         set(Zookeeper_INCLUDE_DIR zookeeper.h ${ZK_INCLUDE_DIR} ${ZK_INCLUDE_DIR_GEN} )
-        find_library(Zookeeper_LIBRARY NAMES zookeeper PATHS ${Zookeeper_LIB_PATHS})
+        find_library(Zookeeper_LIBRARY NAMES ${ZK_LibName} PATHS ${Zookeeper_LIB_PATHS})
     endif()
 else()
     set(Zookeeper_LIB_PATHS /usr/local/lib /opt/local/lib)

http://git-wip-us.apache.org/repos/asf/drill/blob/3a35a420/contrib/native/client/readme.win.txt
----------------------------------------------------------------------
diff --git a/contrib/native/client/readme.win.txt b/contrib/native/client/readme.win.txt
index 35dbb1d..9391038 100644
--- a/contrib/native/client/readme.win.txt
+++ b/contrib/native/client/readme.win.txt
@@ -59,6 +59,7 @@ Windows platforms should be more or less similar.
             boost
             zookeeper C API
             protobufs
+            cppunit
         The Drill client is linked with STATIC versions of these libraries. The
         libraries are themselves linked with the DYNAMIC C Runtime DLLs. It is
         important that the libraries all have the same linkage model, otherwise the
@@ -74,6 +75,8 @@ Windows platforms should be more or less similar.
             this is the directory for the full Zookeeper source not just the
             source for the C library.
             PROTOBUF_HOME - Directory where Protobuf source is installed.
+            CPPUNIT_HOME - Directory where CPPUnit source is installed
+    d) The build assumes that Powershell is installed
 
 2.1 Boost (version 1.55)
     a) Download Boost from:
@@ -133,13 +136,21 @@ Windows platforms should be more or less similar.
     fix that for the 64 bit build, apply patch zookeeper-3.4.6-x64.patch
     For example in Msysgit 
         $ cd <ZOOKEEPER_HOME> && git apply <DRILL_HOME>/contrib/native/client/patches/zookeeper-3.4.6-x64.patch
-    c) InVisual Studio 2010 Express open <ZOOKEEPER_HOME>/src/c/zookeeper.sln
+    c) In Visual Studio 2010 Express open <ZOOKEEPER_HOME>/src/c/zookeeper.sln
         i) Add a 64 bit project configuration for each project. (Make sure the
             platform toolset is set to Windows7.1SDK)
        ii) Change the output type for the zookeeper project to a static lib
             Properties->Configuration Properties->General->Configuration Type = Static Library
       iii) In the cli project add the preprocessor define USE_STATIC_LIB
-      iv) Build. Build zookeeper lib first, then build cli 
+       iv) Build. Build zookeeper lib first, then build cli 
+
+2.4 CppUnit (3.4.6) 
+    a) Download cppunit and unzip/untar it. 
+       Latest version is available at: http://dev-www.libreoffice.org/src/cppunit-1.13.2.tar.gz
+       More informations: https://www.freedesktop.org/wiki/Software/cppunit/
+    b) Set the CPPUNIT_HOME environment variable
+    c) InVisual Studio 2010 Express open <CPPUNIT_HOME>/src/CppUnitLibraries2010.sln
+       i) Build cppunit project
 
 3 Building Drill Clientlib
 3.1 SET the following environment variables
@@ -152,10 +163,10 @@ Windows platforms should be more or less similar.
     C:> cd build
 
     a) For the 32 bit build :
-        C:> cmake -G "Visual Studio 10" -D ZOOKEEPER_HOME=<ZOOKEPER_HOME> -D PROTOBUF_SRC_ROOT_FOLDER=<PROTOBUF_HOME> -D CMAKE_BUILD_TYPE=Debug   ..
+        C:> cmake -G "Visual Studio 10" -D ZOOKEEPER_HOME=<ZOOKEPER_HOME> -D PROTOBUF_SRC_ROOT_FOLDER=<PROTOBUF_HOME> -D CPPUNIT_HOME=<CPPUNIT_HOME> -D CMAKE_BUILD_TYPE=Debug   ..
 
     b) For the 64 bit build :
-        C:> cmake -G "Visual Studio 10 Win64 " -D ZOOKEEPER_HOME=<ZOOKEPER_HOME> -D PROTOBUF_SRC_ROOT_FOLDER=<PROTOBUF_HOME> -D CMAKE_BUILD_TYPE=Debug   ..
+        C:> cmake -G "Visual Studio 10 Win64 " -D ZOOKEEPER_HOME=<ZOOKEPER_HOME> -D PROTOBUF_SRC_ROOT_FOLDER=<PROTOBUF_HOME> -D CPPUNIT_HOME=<CPPUNIT_HOME> -D CMAKE_BUILD_TYPE=Debug   ..
 
 3.3 Open the generated <DRILL_HOME>/contrib/native/client/build/drillclient.sln 
     file in Visual Studio.
@@ -185,4 +196,4 @@ Windows platforms should be more or less similar.
     In particular, for debug builds, check the path of the protobuf library.
 	
 5 Testing with querySubmitter
-querySubmitter query="select * from INFORMAITON_SCHEMA.SCHEMATA" type=sql connectStr=local=192.168.39.43:31010 api=sync logLevel=trace user=yourUserName password=yourPassWord
\ No newline at end of file
+querySubmitter query="select * from INFORMAITON_SCHEMA.SCHEMATA" type=sql connectStr=local=192.168.39.43:31010 api=sync logLevel=trace user=yourUserName password=yourPassWord

http://git-wip-us.apache.org/repos/asf/drill/blob/3a35a420/contrib/native/client/src/clientlib/utils.hpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/clientlib/utils.hpp b/contrib/native/client/src/clientlib/utils.hpp
index 36fb91f..3237aa3 100644
--- a/contrib/native/client/src/clientlib/utils.hpp
+++ b/contrib/native/client/src/clientlib/utils.hpp
@@ -62,7 +62,7 @@ class AllocatedBuffer{
 
 };
 
-class Utils{
+class DECLSPEC_DRILL_CLIENT Utils{
     public:
         static boost::random::random_device s_RNG;   //Truly random (expensive and device dependent)
         static boost::random::mt19937 s_URNG; //Pseudo random with a period of ( 2^19937 - 1 )

http://git-wip-us.apache.org/repos/asf/drill/blob/3a35a420/contrib/native/client/src/test/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/test/CMakeLists.txt b/contrib/native/client/src/test/CMakeLists.txt
new file mode 100644
index 0000000..892b58c
--- /dev/null
+++ b/contrib/native/client/src/test/CMakeLists.txt
@@ -0,0 +1,40 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+# Drill Client unit tests
+set (TESTS_SRC_FILES
+    ${CMAKE_CURRENT_SOURCE_DIR}/UtilsTest.cpp
+    )
+
+find_package(CppUnit REQUIRED)
+include_directories(${CMAKE_CURRENT_SOURCE_DIR} ${CMAKE_CURRENT_SOURCE_DIR}/../include )
+include_directories(${CMAKE_CURRENT_SOURCE_DIR} ${CMAKE_CURRENT_SOURCE_DIR}/../clientlib )
+include_directories(${CPPUNIT_INCLUDE_DIR})
+include_directories(${Boost_INCLUDE_DIRS})
+include_directories(${PROTOBUF_INCLUDE_DIR})
+include_directories(${Zookeeper_INCLUDE_DIRS})
+
+link_directories(/usr/local/lib)
+add_executable(unit-tests ${CMAKE_CURRENT_SOURCE_DIR}/main.cpp ${TESTS_SRC_FILES} )
+target_link_libraries(unit-tests drillClient protomsgs y2038 ${CPPUNIT_LIBRARY} ${Boost_LIBRARIES} ${PROTOBUF_LIBRARY} ${Zookeeper_LIBRARIES})
+
+foreach(testfile ${TESTS_SRC_FILES})
+get_filename_component(testname ${testfile} NAME_WE)
+add_test(NAME ${testname}
+         COMMAND unit-tests ${testname})
+endforeach(testfile)

http://git-wip-us.apache.org/repos/asf/drill/blob/3a35a420/contrib/native/client/src/test/UtilsTest.cpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/test/UtilsTest.cpp b/contrib/native/client/src/test/UtilsTest.cpp
new file mode 100644
index 0000000..0fba45e
--- /dev/null
+++ b/contrib/native/client/src/test/UtilsTest.cpp
@@ -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.
+ */
+
+#include <string>
+
+#include <cppunit/TestFixture.h>
+#include <cppunit/extensions/HelperMacros.h>
+
+#include "utils.hpp"
+
+class UtilsTest: public CppUnit::TestFixture {
+public:
+    UtilsTest() {}
+
+    CPPUNIT_TEST_SUITE( UtilsTest );
+    CPPUNIT_TEST(testParseConnectStr);
+    CPPUNIT_TEST_SUITE_END();
+
+
+    void testParseConnectStr() {
+        std::string protocol;
+        std::string hostAndPort;
+        std::string path;
+
+        Drill::Utils::parseConnectStr("local=localhost:12345/path/to/drill",
+                path,
+                protocol,
+                hostAndPort);
+
+        CPPUNIT_ASSERT(protocol == "local");
+        CPPUNIT_ASSERT(hostAndPort == "localhost:12345");
+        CPPUNIT_ASSERT(path == "/path/to/drill");
+    }
+};
+
+CPPUNIT_TEST_SUITE_REGISTRATION( UtilsTest );

http://git-wip-us.apache.org/repos/asf/drill/blob/3a35a420/contrib/native/client/src/test/main.cpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/test/main.cpp b/contrib/native/client/src/test/main.cpp
new file mode 100644
index 0000000..e5e1710
--- /dev/null
+++ b/contrib/native/client/src/test/main.cpp
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+
+#include <cppunit/extensions/TestFactoryRegistry.h>
+#include <cppunit/ui/text/TestRunner.h>
+
+int main( int argc, char **argv)
+{
+    CppUnit::TextUi::TestRunner runner;
+    CppUnit::TestFactoryRegistry& registry = CppUnit::TestFactoryRegistry::getRegistry();
+
+    CppUnit::Test* testSuite = registry.makeTest();
+    CppUnit::Test* test;
+    if (argc > 1) {
+        test = testSuite->findTest(argv[1]);
+    }
+    else {
+        test = testSuite;
+    }
+
+    runner.addTest( testSuite );
+    bool wasSuccessful = runner.run("", false );
+    return !wasSuccessful;
+}


[11/15] drill git commit: DRILL-4420: C++ API for metadata access and prepared statements

Posted by pa...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/166c4ce7/contrib/native/client/src/clientlib/metadata.hpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/clientlib/metadata.hpp b/contrib/native/client/src/clientlib/metadata.hpp
new file mode 100644
index 0000000..0cc8987
--- /dev/null
+++ b/contrib/native/client/src/clientlib/metadata.hpp
@@ -0,0 +1,288 @@
+/*
+ * 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.
+ */
+
+
+#ifndef DRILL_METADATA_H
+#define DRILL_METADATA_H
+
+#include <boost/ref.hpp>
+
+#include "drill/common.hpp"
+#include "drill/drillClient.hpp"
+#include "env.h"
+#include "User.pb.h"
+
+namespace Drill {
+class DrillClientImpl;
+
+namespace meta {
+	class DrillCatalogMetadata: public meta::CatalogMetadata {
+	public:
+		DrillCatalogMetadata(const ::exec::user::CatalogMetadata& metadata):
+			meta::CatalogMetadata(),
+			m_pMetadata(metadata){
+		}
+
+	  bool hasCatalogName() const { return m_pMetadata.get().has_catalog_name(); }
+	  const std::string& getCatalogName() const { return m_pMetadata.get().catalog_name(); }
+
+	  bool hasDescription() const { return m_pMetadata.get().has_description(); }
+	  const std::string& getDescription() const { return m_pMetadata.get().description(); }
+
+	  bool hasConnect() const { return m_pMetadata.get().has_connect(); }
+	  const std::string& getConnect() const { return m_pMetadata.get().connect(); }
+
+	private:
+		boost::reference_wrapper<const ::exec::user::CatalogMetadata> m_pMetadata;
+	};
+
+	class DrillSchemaMetadata: public meta::SchemaMetadata {
+	public:
+		DrillSchemaMetadata(const ::exec::user::SchemaMetadata& metadata):
+			meta::SchemaMetadata(),
+			m_pMetadata(metadata){
+		}
+
+		bool hasCatalogName() const { return m_pMetadata.get().has_catalog_name(); }
+		const std::string& getCatalogName() const { return m_pMetadata.get().catalog_name(); }
+
+		bool hasSchemaName() const { return m_pMetadata.get().has_schema_name(); }
+		const std::string& getSchemaName() const { return m_pMetadata.get().schema_name(); }
+
+		bool hasOwnerName() const { return m_pMetadata.get().has_owner(); }
+		const std::string& getOwner() const { return m_pMetadata.get().owner(); }
+
+		bool hasType() const { return m_pMetadata.get().has_type(); }
+		const std::string& getType() const { return m_pMetadata.get().type(); }
+
+		bool hasMutable() const { return m_pMetadata.get().has_mutable_(); }
+		const std::string& getMutable() const { return m_pMetadata.get().mutable_(); }
+
+	private:
+		boost::reference_wrapper<const ::exec::user::SchemaMetadata> m_pMetadata;
+	};
+
+	class DrillTableMetadata: public meta::TableMetadata {
+	public:
+		DrillTableMetadata(const ::exec::user::TableMetadata& metadata):
+			meta::TableMetadata(),
+			m_pMetadata(metadata){
+		}
+
+	  bool hasCatalogName() const { return m_pMetadata.get().has_catalog_name(); }
+	  const std::string& getCatalogName() const { return m_pMetadata.get().catalog_name(); }
+
+	  bool hasSchemaName() const { return m_pMetadata.get().has_schema_name(); }
+	  const std::string& getSchemaName() const { return m_pMetadata.get().schema_name(); }
+
+	  bool hasTableName() const { return m_pMetadata.get().has_table_name(); }
+	  const std::string& getTableName() const { return m_pMetadata.get().table_name(); }
+
+	  bool hasType() const { return m_pMetadata.get().has_type(); }
+	  const std::string& getType() const { return m_pMetadata.get().type(); }
+
+	private:
+	  boost::reference_wrapper<const ::exec::user::TableMetadata> m_pMetadata;
+	};
+
+	class DrillColumnMetadata: public meta::ColumnMetadata {
+	public:
+		DrillColumnMetadata(const ::exec::user::ColumnMetadata& metadata):
+			meta::ColumnMetadata(),
+			m_pMetadata(metadata){
+		}
+
+		bool hasCatalogName() const { return m_pMetadata.get().has_catalog_name(); }
+		const std::string& getCatalogName() const { return m_pMetadata.get().catalog_name(); }
+
+		bool hasSchemaName() const { return m_pMetadata.get().has_schema_name(); }
+		const std::string& getSchemaName() const { return m_pMetadata.get().schema_name(); }
+
+		bool hasTableName() const { return m_pMetadata.get().has_table_name(); }
+		const std::string& getTableName() const { return m_pMetadata.get().table_name(); }
+
+		bool hasColumnName() const { return m_pMetadata.get().has_column_name(); }
+		const std::string& getColumnName() const { return m_pMetadata.get().column_name(); }
+
+		bool hasOrdinalPosition() const { return m_pMetadata.get().has_ordinal_position(); }
+		std::size_t getOrdinalPosition() const { return m_pMetadata.get().ordinal_position(); }
+
+		bool hasDefaultValue() const { return m_pMetadata.get().has_default_value(); }
+		const std::string& getDefaultValue() const { return m_pMetadata.get().default_value(); }
+
+		bool hasNullable() const { return m_pMetadata.get().has_is_nullable(); }
+		bool isNullable() const { return m_pMetadata.get().is_nullable(); }
+
+		bool hasDataType() const { return m_pMetadata.get().has_data_type(); }
+		const std::string& getDataType() const { return m_pMetadata.get().data_type(); }
+
+		bool hasColumnSize() const { return m_pMetadata.get().has_column_size(); }
+		std::size_t getColumnSize() const { return m_pMetadata.get().column_size(); }
+
+		bool hasCharMaxLength() const { return m_pMetadata.get().has_char_max_length(); }
+		std::size_t getCharMaxLength() const { return m_pMetadata.get().char_max_length(); }
+
+		bool hasCharOctetLength() const { return m_pMetadata.get().has_char_octet_length(); }
+		std::size_t getCharOctetLength() const { return m_pMetadata.get().char_octet_length(); }
+
+		bool hasNumericPrecision() const { return m_pMetadata.get().has_numeric_precision(); }
+		int32_t getNumericPrecision() const { return m_pMetadata.get().numeric_precision(); }
+
+		bool hasNumericRadix() const { return m_pMetadata.get().has_numeric_precision_radix(); }
+		int32_t getNumericRadix() const { return m_pMetadata.get().numeric_precision_radix(); }
+
+		bool hasNumericScale() const { return m_pMetadata.get().has_numeric_scale(); }
+		int32_t getNumericScale() const { return m_pMetadata.get().numeric_scale(); }
+
+		bool hasIntervalType() const { return m_pMetadata.get().has_interval_type(); }
+		const std::string& getIntervalType() const { return m_pMetadata.get().interval_type(); }
+
+		bool hasIntervalPrecision() const { return m_pMetadata.get().has_interval_precision(); }
+		int32_t getIntervalPrecision() const { return m_pMetadata.get().interval_precision(); }
+
+	private:
+		boost::reference_wrapper<const ::exec::user::ColumnMetadata> m_pMetadata;
+	};
+
+    class DrillMetadata: public Metadata {
+    public:
+        static const std::string s_connectorName; 
+        static const std::string s_connectorVersion; 
+
+        static const std::string s_serverName;
+        static const std::string s_serverVersion;
+
+        static const std::string s_catalogSeparator;
+        static const std::string s_catalogTerm;
+
+        static const std::string s_identifierQuoteString;
+        static const std::vector<std::string> s_sqlKeywords;
+        static const std::vector<std::string> s_numericFunctions;
+        static const std::string s_schemaTerm;
+        static const std::string s_searchEscapeString;
+        static const std::string s_specialCharacters;
+        static const std::vector<std::string> s_stringFunctions;
+        static const std::vector<std::string> s_systemFunctions;
+        static const std::string s_tableTerm;
+        static const std::vector<std::string> s_dateTimeFunctions;
+
+        DrillMetadata(DrillClientImpl& client): Metadata(), m_client(client) {}
+        ~DrillMetadata() {}
+
+        DrillClientImpl& client() { return m_client; }
+
+        const std::string& getConnectorName() const { return s_connectorName; };
+        const std::string& getConnectorVersion() const { return s_connectorVersion; }
+        uint32_t getConnectorMajorVersion() const { return DRILL_VERSION_MAJOR; } 
+        uint32_t getConnectorMinorVersion() const { return DRILL_VERSION_MINOR; } 
+        uint32_t getConnectorPatchVersion() const { return DRILL_VERSION_PATCH; } 
+
+        const std::string& getServerName() const;
+        const std::string& getServerVersion() const;
+        uint32_t getServerMajorVersion() const;
+        uint32_t getServerMinorVersion() const;
+        uint32_t getServerPatchVersion() const;
+
+        status_t getCatalogs(const std::string& catalogPattern, Metadata::pfnCatalogMetadataListener listener, void* listenerCtx, QueryHandle_t* qHandle);
+        status_t getSchemas(const std::string& catalogPattern, const std::string& schemaPattern, Metadata::pfnSchemaMetadataListener listener, void* listenerCtx, QueryHandle_t* qHandle);
+        status_t getTables(const std::string& catalogPattern, const std::string& schemaPattern, const std::string& tablePattern, const std::vector<std::string>* tableTypes, Metadata::pfnTableMetadataListener listener, void* listenerCtx, QueryHandle_t* qHandle);
+        status_t getColumns(const std::string& catalogPattern, const std::string& schemaPattern, const std:: string& tablePattern, const std::string& columnPattern, Metadata::pfnColumnMetadataListener listener, void* listenerCtx, QueryHandle_t* qHandle);
+
+        bool areAllTableSelectable() const { return false; }
+        bool isCatalogAtStart() const { return true; }
+        const std::string& getCatalogSeparator() const { return s_catalogSeparator; }
+        const std::string& getCatalogTerm() const { return s_catalogTerm; }
+        bool isColumnAliasingSupported() const { return true; }
+        bool isNullPlusNonNullNull() const { return true; }
+        bool isConvertSupported(common::MinorType from, common::MinorType to) const;
+        meta::CorrelationNamesSupport getCorrelationNames() const { return meta::CN_ANY_NAMES; }
+        bool isReadOnly() const { return false; }
+        meta::DateTimeLiteralSupport getDateTimeLiteralsSupport() const {
+            return DL_DATE
+                | DL_TIME
+                | DL_TIMESTAMP
+                | DL_INTERVAL_YEAR
+                | DL_INTERVAL_MONTH
+                | DL_INTERVAL_DAY
+                | DL_INTERVAL_HOUR
+                | DL_INTERVAL_MINUTE
+                | DL_INTERVAL_SECOND
+                | DL_INTERVAL_YEAR_TO_MONTH
+                | DL_INTERVAL_DAY_TO_HOUR
+                | DL_INTERVAL_DAY_TO_MINUTE
+                | DL_INTERVAL_DAY_TO_SECOND
+                | DL_INTERVAL_HOUR_TO_MINUTE
+                | DL_INTERVAL_HOUR_TO_SECOND
+                | DL_INTERVAL_MINUTE_TO_SECOND;
+        }
+
+        meta::CollateSupport getCollateSupport() const { return meta::C_NONE; }// supported?
+        meta::GroupBySupport getGroupBySupport() const { return meta::GB_UNRELATED; }
+        meta::IdentifierCase getIdentifierCase() const { return meta::IC_STORES_UPPER; } // to check?
+
+        const std::string& getIdentifierQuoteString() const { return s_identifierQuoteString; }
+        const std::vector<std::string>& getSQLKeywords() const { return s_sqlKeywords; }
+        bool isLikeEscapeClauseSupported() const { return true; }
+        std::size_t getMaxBinaryLiteralLength() const { return 0; }
+        std::size_t getMaxCatalogNameLength() const { return 0; }
+        std::size_t getMaxCharLiteralLength() const { return 0; }
+        std::size_t getMaxColumnNameLength() const { return 0; }
+        std::size_t getMaxColumnsInGroupBy() const { return 0; }
+        std::size_t getMaxColumnsInOrderBy() const { return 0; }
+        std::size_t getMaxColumnsInSelect() const { return 0; }
+        std::size_t getMaxCursorNameLength() const { return 0; }
+        std::size_t getMaxLogicalLobSize() const { return 0; }
+        std::size_t getMaxStatements() const { return 0; }
+        std::size_t getMaxRowSize() const { return 0; }
+        bool isBlobIncludedInMaxRowSize() const { return true; }
+        std::size_t getMaxSchemaNameLength() const { return 0; }
+        std::size_t getMaxStatementLength() const { return 0; }
+        std::size_t getMaxTableNameLength() const { return 0; }
+        std::size_t getMaxTablesInSelect() const { return 0; }
+        std::size_t getMaxUserNameLength() const { return 0; }
+        meta::NullCollation getNullCollation() const { return meta::NC_AT_END; }
+        const std::vector<std::string>& getNumericFunctions() const { return s_numericFunctions; }
+        meta::OuterJoinSupport getOuterJoinSupport() const { return meta::OJ_LEFT 
+            | meta::OJ_RIGHT 
+            | meta::OJ_FULL;
+        }
+        bool isUnrelatedColumnsInOrderBySupported() const { return true; }
+        meta::QuotedIdentifierCase getQuotedIdentifierCase() const { return meta::QIC_SUPPORTS_MIXED; }
+        const std::string& getSchemaTerm() const { return s_schemaTerm; }
+        const std::string& getSearchEscapeString() const { return s_searchEscapeString; }
+        const std::string& getSpecialCharacters() const { return s_specialCharacters; }
+        const std::vector<std::string>& getStringFunctions() const { return s_stringFunctions; }
+        meta::SubQuerySupport getSubQuerySupport() const { return SQ_CORRELATED
+                | SQ_IN_COMPARISON
+                | SQ_IN_EXISTS
+                | SQ_IN_QUANTIFIED;
+        }
+        const std::vector<std::string>& getSystemFunctions() const { return s_systemFunctions; }
+        const std::string& getTableTerm() const { return s_tableTerm; }
+        const std::vector<std::string>& getDateTimeFunctions() const { return s_dateTimeFunctions; }
+        bool isTransactionSupported() const { return false; }
+        meta::UnionSupport getUnionSupport() const { return meta::U_UNION | meta::U_UNION_ALL; }
+        bool isSelectForUpdateSupported() const { return false; }
+
+    private:
+        DrillClientImpl& m_client;
+    };
+} // namespace meta
+} // namespace Drill
+
+#endif // DRILL_METADATA

http://git-wip-us.apache.org/repos/asf/drill/blob/166c4ce7/contrib/native/client/src/clientlib/recordBatch.cpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/clientlib/recordBatch.cpp b/contrib/native/client/src/clientlib/recordBatch.cpp
index c6c033b..6e13293 100644
--- a/contrib/native/client/src/clientlib/recordBatch.cpp
+++ b/contrib/native/client/src/clientlib/recordBatch.cpp
@@ -17,6 +17,7 @@
  */
 
 #include "drill/common.hpp"
+#include "drill/fieldmeta.hpp"
 #include "drill/recordBatch.hpp"
 #include "utils.hpp"
 #include "../protobuf/User.pb.h"
@@ -403,17 +404,6 @@ bool RecordBatch::isLastChunk(){
 
 
 
-void FieldMetadata::set(const exec::shared::SerializedField& f){
-    m_name=f.name_part().name();
-    m_minorType=f.major_type().minor_type();
-    m_dataMode=f.major_type().mode();
-    m_valueCount=f.value_count();
-    m_scale=f.major_type().scale();
-    m_precision=f.major_type().precision();
-    m_bufferLength=f.buffer_length();
-}
-
-
 void DateHolder::load(){
     m_year=1970;
     m_month=1;

http://git-wip-us.apache.org/repos/asf/drill/blob/166c4ce7/contrib/native/client/src/clientlib/rpcDecoder.cpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/clientlib/rpcDecoder.cpp b/contrib/native/client/src/clientlib/rpcDecoder.cpp
deleted file mode 100644
index d3cf50c..0000000
--- a/contrib/native/client/src/clientlib/rpcDecoder.cpp
+++ /dev/null
@@ -1,153 +0,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.
- */
-
-
-#include <iostream>
-#include <google/protobuf/io/coded_stream.h>
-#include "drill/common.hpp"
-#include "rpcEncoder.hpp"
-#include "rpcDecoder.hpp"
-#include "rpcMessage.hpp"
-
-namespace Drill{
-
-// return the number of bytes we have read
-int RpcDecoder::LengthDecode(const uint8_t* buf, uint32_t* p_length) {
-
-    using google::protobuf::io::CodedInputStream;
-
-    // read the frame to get the length of the message and then
-
-    CodedInputStream* cis = new CodedInputStream(buf, 5); // read 5 bytes at most
-
-    int pos0 = cis->CurrentPosition(); // for debugging
-    cis->ReadVarint32(p_length);
-
-    #ifdef CODER_DEBUG
-    cerr << "p_length = " << *p_length << endl;
-    #endif
-
-    int pos1 = cis->CurrentPosition();
-
-    #ifdef CODER_DEBUG
-    cerr << "Reading full length " << *p_length << endl;
-    #endif
-    assert( (pos1-pos0) == getRawVarintSize(*p_length));
-    delete cis;
-    return (pos1-pos0);
-}
-
-// TODO: error handling
-//
-// - assume that the entire message is in the buffer and the buffer is constrained to this message
-// - easy to handle with raw arry in C++
-int RpcDecoder::Decode(const uint8_t* buf, int length, InBoundRpcMessage& msg) {
-    using google::protobuf::io::CodedInputStream;
-
-    // if(!ctx.channel().isOpen()){ return; }
-
-    #ifdef  EXTRA_DEBUGGING
-    std::cerr <<  "\nInbound rpc message received." << std::endl;
-    #endif
-
-    CodedInputStream* cis = new CodedInputStream(buf, length);
-
-
-    int pos0 = cis->CurrentPosition(); // for debugging
-
-    int len_limit = cis->PushLimit(length);
-
-    uint32_t header_length = 0;
-    cis->ExpectTag(RpcEncoder::HEADER_TAG);
-    cis->ReadVarint32(&header_length);
-
-    #ifdef CODER_DEBUG
-    cerr << "Reading header length " << header_length << ", post read index " << cis->CurrentPosition() << endl;
-    #endif
-
-    exec::rpc::RpcHeader header;
-    int header_limit = cis->PushLimit(header_length);
-    header.ParseFromCodedStream(cis);
-    cis->PopLimit(header_limit);
-    msg.m_has_mode = header.has_mode();
-    msg.m_mode = header.mode();
-    msg.m_coord_id = header.coordination_id();
-    msg.m_has_rpc_type = header.has_rpc_type();
-    msg.m_rpc_type = header.rpc_type();
-
-    //if(RpcConstants.EXTRA_DEBUGGING) logger.debug(" post header read index {}", buffer.readerIndex());
-
-    // read the protobuf body into a buffer.
-    cis->ExpectTag(RpcEncoder::PROTOBUF_BODY_TAG);
-    uint32_t p_body_length = 0;
-    cis->ReadVarint32(&p_body_length);
-
-    #ifdef CODER_DEBUG
-    cerr << "Reading protobuf body length " << p_body_length << ", post read index " << cis->CurrentPosition() << endl;
-    #endif
-
-    msg.m_pbody.resize(p_body_length);
-    cis->ReadRaw(msg.m_pbody.data(),p_body_length);
-
-
-    // read the data body.
-    if (cis->BytesUntilLimit() > 0 ) {
-    #ifdef CODER_DEBUG
-        cerr << "Reading raw body, buffer has "<< cis->BytesUntilLimit() << " bytes available, current possion "<< cis->CurrentPosition()  << endl;
-    #endif
-        cis->ExpectTag(RpcEncoder::RAW_BODY_TAG);
-        uint32_t d_body_length = 0;
-        cis->ReadVarint32(&d_body_length);
-
-        if(cis->BytesUntilLimit() != d_body_length) {
-    #ifdef CODER_DEBUG
-            cerr << "Expected to receive a raw body of " << d_body_length << " bytes but received a buffer with " <<cis->BytesUntilLimit() << " bytes." << endl;
-    #endif
-        }
-        //msg.m_dbody.resize(d_body_length);
-        //cis->ReadRaw(msg.m_dbody.data(), d_body_length);
-        uint32_t currPos=cis->CurrentPosition();
-        cis->GetDirectBufferPointer((const void**)&msg.m_dbody, (int*)&d_body_length);
-        assert(msg.m_dbody==buf+currPos);
-        cis->Skip(d_body_length);
-    #ifdef CODER_DEBUG
-        cerr << "Read raw body of " << d_body_length << " bytes" << endl;
-    #endif
-    } else {
-    #ifdef CODER_DEBUG
-        cerr << "No need to read raw body, no readable bytes left." << endl;
-    #endif
-    }
-    cis->PopLimit(len_limit);
-
-
-    // return the rpc message.
-    // move the reader index forward so the next rpc call won't try to work with it.
-    // buffer.skipBytes(dBodyLength);
-    // messageCounter.incrementAndGet();
-    #ifdef CODER_DEBUG
-    cerr << "Inbound Rpc Message Decoded " << msg << endl;
-    #endif
-
-    int pos1 = cis->CurrentPosition();
-    assert((pos1-pos0) == length);
-    delete cis;
-    return (pos1-pos0);
-}
-
-}//namespace Drill

http://git-wip-us.apache.org/repos/asf/drill/blob/166c4ce7/contrib/native/client/src/clientlib/rpcDecoder.hpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/clientlib/rpcDecoder.hpp b/contrib/native/client/src/clientlib/rpcDecoder.hpp
deleted file mode 100644
index dca49f7..0000000
--- a/contrib/native/client/src/clientlib/rpcDecoder.hpp
+++ /dev/null
@@ -1,38 +0,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.
- */
-
-
-#ifndef RPC_DECODER_H
-#define RPC_DECODER_H
-
-#include "rpcMessage.hpp"
-
-namespace Drill {
-
-class RpcDecoder {
-    public:
-        RpcDecoder() { }
-        ~RpcDecoder() { }
-        // bool Decode(const DataBuf& buf);
-        // bool Decode(const DataBuf& buf, InBoundRpcMessage& msg);
-        static int LengthDecode(const uint8_t* buf, uint32_t* length); // read the length prefix (at most 4 bytes)
-        static int Decode(const uint8_t* buf, int length, InBoundRpcMessage& msg);
-};
-
-} // namespace Drill
-#endif

http://git-wip-us.apache.org/repos/asf/drill/blob/166c4ce7/contrib/native/client/src/clientlib/rpcEncoder.cpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/clientlib/rpcEncoder.cpp b/contrib/native/client/src/clientlib/rpcEncoder.cpp
deleted file mode 100644
index 2f354d7..0000000
--- a/contrib/native/client/src/clientlib/rpcEncoder.cpp
+++ /dev/null
@@ -1,109 +0,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.
- */
-
-
-#include <google/protobuf/io/coded_stream.h>
-#include <google/protobuf/io/zero_copy_stream_impl_lite.h>
-#include <google/protobuf/message_lite.h>
-#include <google/protobuf/wire_format_lite.h>
-
-#include "drill/common.hpp"
-#include "rpcEncoder.hpp"
-#include "rpcMessage.hpp"
-
-namespace Drill{
-
-using google::protobuf::internal::WireFormatLite;
-using exec::rpc::CompleteRpcMessage;
-
-const uint32_t RpcEncoder::HEADER_TAG = WireFormatLite::MakeTag(CompleteRpcMessage::kHeaderFieldNumber, WireFormatLite::WIRETYPE_LENGTH_DELIMITED);
-const uint32_t RpcEncoder::PROTOBUF_BODY_TAG = WireFormatLite::MakeTag(CompleteRpcMessage::kProtobufBodyFieldNumber, WireFormatLite::WIRETYPE_LENGTH_DELIMITED);
-const uint32_t RpcEncoder::RAW_BODY_TAG = WireFormatLite::MakeTag(CompleteRpcMessage::kRawBodyFieldNumber, WireFormatLite::WIRETYPE_LENGTH_DELIMITED);
-const uint32_t RpcEncoder::HEADER_TAG_LENGTH = getRawVarintSize(HEADER_TAG);
-const uint32_t RpcEncoder::PROTOBUF_BODY_TAG_LENGTH = getRawVarintSize(PROTOBUF_BODY_TAG);
-const uint32_t RpcEncoder::RAW_BODY_TAG_LENGTH = getRawVarintSize(RAW_BODY_TAG);
-
-
-bool RpcEncoder::Encode(DataBuf& buf, OutBoundRpcMessage& msg) {
-    using exec::rpc::RpcHeader;
-    using google::protobuf::io::CodedOutputStream;
-    using google::protobuf::io::ArrayOutputStream;
-    // Todo:
-    //
-    // - let a context manager to allocate a buffer `ByteBuf buf = ctx.alloc().buffer();`
-    // - builder pattern
-    //
-    #ifdef CODER_DEBUG
-    cerr << "\nEncoding outbound message " << msg << endl;
-    #endif
-
-    RpcHeader header;
-    header.set_mode(msg.m_mode);
-    header.set_coordination_id(msg.m_coord_id);
-    header.set_rpc_type(msg.m_rpc_type);
-
-    // calcute the length of the message
-    int header_length = header.ByteSize();
-    int proto_body_length = msg.m_pbody->ByteSize();
-    int full_length = HEADER_TAG_LENGTH + getRawVarintSize(header_length) + header_length + \
-                      PROTOBUF_BODY_TAG_LENGTH + getRawVarintSize(proto_body_length) + proto_body_length;
-
-    /*
-       if(raw_body_length > 0) {
-       full_length += (RAW_BODY_TAG_LENGTH + getRawVarintSize(raw_body_length) + raw_body_length);
-       }
-       */
-
-    buf.resize(full_length + getRawVarintSize(full_length));
-    ArrayOutputStream* os = new ArrayOutputStream(buf.data(), buf.size());
-    CodedOutputStream* cos = new CodedOutputStream(os);
-
-
-    #ifdef CODER_DEBUG
-    cerr << "Writing full length " << full_length << endl;
-    #endif
-
-    // write full length first (this is length delimited stream).
-    cos->WriteVarint32(full_length);
-
-    #ifdef CODER_DEBUG
-    cerr << "Writing header length " << header_length << endl;
-    #endif
-
-    cos->WriteVarint32(HEADER_TAG);
-    cos->WriteVarint32(header_length);
-
-    header.SerializeToCodedStream(cos);
-
-    // write protobuf body length and body
-    #ifdef CODER_DEBUG
-    cerr << "Writing protobuf body length " << proto_body_length << endl;
-    #endif
-
-    cos->WriteVarint32(PROTOBUF_BODY_TAG);
-    cos->WriteVarint32(proto_body_length);
-    msg.m_pbody->SerializeToCodedStream(cos);
-
-    delete os;
-    delete cos;
-
-    // Done! no read to write data body for client
-    return true;
-}
-
-} // namespace Drill

http://git-wip-us.apache.org/repos/asf/drill/blob/166c4ce7/contrib/native/client/src/clientlib/rpcEncoder.hpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/clientlib/rpcEncoder.hpp b/contrib/native/client/src/clientlib/rpcEncoder.hpp
deleted file mode 100644
index a4a7216..0000000
--- a/contrib/native/client/src/clientlib/rpcEncoder.hpp
+++ /dev/null
@@ -1,55 +0,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.
- */
-
-
-#ifndef RPC_ENCODER_H
-#define RPC_ENCODER_H
-
-#include "rpcMessage.hpp"
-
-namespace Drill {
-
-class RpcEncoder {
-    public:
-        RpcEncoder() {}
-        ~RpcEncoder() { }
-        bool Encode(DataBuf& buf,OutBoundRpcMessage& msg);
-        static const uint32_t HEADER_TAG;
-        static const uint32_t PROTOBUF_BODY_TAG;
-        static const uint32_t RAW_BODY_TAG;
-        static const uint32_t HEADER_TAG_LENGTH;
-        static const uint32_t PROTOBUF_BODY_TAG_LENGTH;
-        static const uint32_t RAW_BODY_TAG_LENGTH;
-};
-
-// copy from java code
-inline int getRawVarintSize(uint32_t value) {
-    int count = 0;
-    while (true) {
-        if ((value & ~0x7F) == 0) {
-            count++;
-            return count;
-        } else {
-            count++;
-            value >>= 7;
-        }
-    }
-}
-
-} // namespace Drill
-#endif

http://git-wip-us.apache.org/repos/asf/drill/blob/166c4ce7/contrib/native/client/src/clientlib/rpcMessage.cpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/clientlib/rpcMessage.cpp b/contrib/native/client/src/clientlib/rpcMessage.cpp
new file mode 100644
index 0000000..13cd7a8
--- /dev/null
+++ b/contrib/native/client/src/clientlib/rpcMessage.cpp
@@ -0,0 +1,241 @@
+/*
+ * 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.
+ */
+
+
+#include <google/protobuf/io/coded_stream.h>
+#include <google/protobuf/io/zero_copy_stream_impl_lite.h>
+#include <google/protobuf/message_lite.h>
+#include <google/protobuf/wire_format_lite.h>
+
+#include "drill/common.hpp"
+#include "rpcMessage.hpp"
+
+namespace Drill{
+namespace rpc {
+
+
+namespace {
+using google::protobuf::internal::WireFormatLite;
+using google::protobuf::io::CodedOutputStream;
+using exec::rpc::CompleteRpcMessage;
+
+static const uint32_t HEADER_TAG = WireFormatLite::MakeTag(CompleteRpcMessage::kHeaderFieldNumber, WireFormatLite::WIRETYPE_LENGTH_DELIMITED);
+static const uint32_t PROTOBUF_BODY_TAG = WireFormatLite::MakeTag(CompleteRpcMessage::kProtobufBodyFieldNumber, WireFormatLite::WIRETYPE_LENGTH_DELIMITED);
+static const uint32_t RAW_BODY_TAG = WireFormatLite::MakeTag(CompleteRpcMessage::kRawBodyFieldNumber, WireFormatLite::WIRETYPE_LENGTH_DELIMITED);
+static const uint32_t HEADER_TAG_LENGTH = CodedOutputStream::VarintSize32(HEADER_TAG);
+static const uint32_t PROTOBUF_BODY_TAG_LENGTH = CodedOutputStream::VarintSize32(PROTOBUF_BODY_TAG);
+}
+
+std::size_t lengthDecode(const uint8_t* buf, uint32_t& length) {
+    using google::protobuf::io::CodedInputStream;
+    using google::protobuf::io::CodedOutputStream;
+
+    // read the frame to get the length of the message and then
+
+    CodedInputStream cis(buf, 5); // read 5 bytes at most
+
+    int startPos(cis.CurrentPosition()); // for debugging
+    if (!cis.ReadVarint32(&length)) {
+    	return -1;
+    }
+
+    #ifdef CODER_DEBUG
+    std::cerr << "length = " << length << std::endl;
+    #endif
+
+    int endPos(cis.CurrentPosition());
+
+    assert((endPos-startPos) == CodedOutputStream::VarintSize32(length));
+    return (endPos-startPos);
+}
+
+// TODO: error handling
+//
+// - assume that the entire message is in the buffer and the buffer is constrained to this message
+// - easy to handle with raw array in C++
+bool decode(const uint8_t* buf, int length, InBoundRpcMessage& msg) {
+    using google::protobuf::io::CodedInputStream;
+
+    CodedInputStream cis(buf, length);
+
+    int startPos(cis.CurrentPosition()); // for debugging
+
+    CodedInputStream::Limit len_limit(cis.PushLimit(length));
+
+    uint32_t header_length(0);
+
+    if (!cis.ExpectTag(HEADER_TAG)) {
+    	return false;
+    }
+
+    if (!cis.ReadVarint32(&header_length)) {
+    	return false;
+    }
+
+    #ifdef CODER_DEBUG
+    std::cerr << "Reading header length " << header_length << ", post read index " << cis.CurrentPosition() << std::endl;
+    #endif
+
+    exec::rpc::RpcHeader header;
+    CodedInputStream::Limit header_limit(cis.PushLimit(header_length));
+
+    if (!header.ParseFromCodedStream(&cis)) {
+    	return false;
+    }
+    cis.PopLimit(header_limit);
+
+    msg.m_has_mode = header.has_mode();
+    msg.m_mode = header.mode();
+    msg.m_coord_id = header.coordination_id();
+    msg.m_has_rpc_type = header.has_rpc_type();
+    msg.m_rpc_type = header.rpc_type();
+
+    // read the protobuf body into a buffer.
+    if (!cis.ExpectTag(PROTOBUF_BODY_TAG)) {
+    	return false;
+    }
+
+    uint32_t pbody_length(0);
+    if (!cis.ReadVarint32(&pbody_length)) {
+    	return false;
+    }
+
+    #ifdef CODER_DEBUG
+    std::cerr << "Reading protobuf body length " << pbody_length << ", post read index " << cis.CurrentPosition() << std::endl;
+    #endif
+
+    msg.m_pbody.resize(pbody_length);
+    if (!cis.ReadRaw(msg.m_pbody.data(), pbody_length)) {
+    	return false;
+    }
+
+    // read the data body.
+    if (cis.BytesUntilLimit() > 0 ) {
+		#ifdef CODER_DEBUG
+			std::cerr << "Reading raw body, buffer has "<< std::cis->BytesUntilLimit() << " bytes available, current possion "<< cis.CurrentPosition()  << endl;
+		#endif
+        if (!cis.ExpectTag(RAW_BODY_TAG)) {
+        	return false;
+        }
+
+        uint32_t dbody_length = 0;
+        if (!cis.ReadVarint32(&dbody_length)) {
+        	return false;
+        }
+
+        if(cis.BytesUntilLimit() != dbody_length) {
+			#ifdef CODER_DEBUG
+					cerr << "Expected to receive a raw body of " << dbody_length << " bytes but received a buffer with " <<cis->BytesUntilLimit() << " bytes." << endl;
+			#endif
+			return false;
+        }
+
+        int currPos(cis.CurrentPosition());
+        int size;
+        cis.GetDirectBufferPointer(const_cast<const void**>(reinterpret_cast<void**>(&msg.m_dbody)), &size);
+        cis.Skip(size);
+
+        assert(dbody_length == size);
+        assert(msg.m_dbody==buf+currPos);
+		#ifdef CODER_DEBUG
+			cerr << "Read raw body of " << dbody_length << " bytes" << endl;
+		#endif
+    } else {
+		#ifdef CODER_DEBUG
+			cerr << "No need to read raw body, no readable bytes left." << endl;
+		#endif
+    }
+    cis.PopLimit(len_limit);
+
+
+    // return the rpc message.
+    // move the reader index forward so the next rpc call won't try to work with it.
+    // buffer.skipBytes(dBodyLength);
+    // messageCounter.incrementAndGet();
+    #ifdef CODER_DEBUG
+    std::cerr << "Inbound Rpc Message Decoded " << msg << std::endl;
+    #endif
+
+    int endPos = cis.CurrentPosition();
+    assert((endPos-startPos) == length);
+    return true;
+}
+
+
+bool encode(DataBuf& buf, const OutBoundRpcMessage& msg) {
+    using exec::rpc::RpcHeader;
+    using google::protobuf::io::CodedOutputStream;
+    // Todo:
+    //
+    // - let a context manager to allocate a buffer `ByteBuf buf = ctx.alloc().buffer();`
+    // - builder pattern
+    //
+    #ifdef CODER_DEBUG
+    std::cerr << "Encoding outbound message " << msg << std::endl;
+    #endif
+
+    RpcHeader header;
+    header.set_mode(msg.m_mode);
+    header.set_coordination_id(msg.m_coord_id);
+    header.set_rpc_type(msg.m_rpc_type);
+
+    // calcute the length of the message
+    int header_length = header.ByteSize();
+    int proto_body_length = msg.m_pbody->ByteSize();
+    int full_length = HEADER_TAG_LENGTH + CodedOutputStream::VarintSize32(header_length) + header_length + \
+                      PROTOBUF_BODY_TAG_LENGTH + CodedOutputStream::VarintSize32(proto_body_length) + proto_body_length;
+
+    /*
+       if(raw_body_length > 0) {
+       full_length += (RAW_BODY_TAG_LENGTH + getRawVarintSize(raw_body_length) + raw_body_length);
+       }
+       */
+
+    buf.resize(full_length + CodedOutputStream::VarintSize32(full_length));
+
+    uint8_t* data = buf.data();
+
+    #ifdef CODER_DEBUG
+    std::cerr << "Writing full length " << full_length << std::endl;
+    #endif
+
+    data = CodedOutputStream::WriteVarint32ToArray(full_length, data);
+
+    #ifdef CODER_DEBUG
+    std::cerr << "Writing header length " << header_length << std::endl;
+    #endif
+
+    data = CodedOutputStream::WriteVarint32ToArray(HEADER_TAG, data);
+    data = CodedOutputStream::WriteVarint32ToArray(header_length, data);
+
+    data = header.SerializeWithCachedSizesToArray(data);
+
+    // write protobuf body length and body
+    #ifdef CODER_DEBUG
+    std::cerr << "Writing protobuf body length " << proto_body_length << std::endl;
+    #endif
+
+    data = CodedOutputStream::WriteVarint32ToArray(PROTOBUF_BODY_TAG, data);
+    data = CodedOutputStream::WriteVarint32ToArray(proto_body_length, data);
+    msg.m_pbody->SerializeWithCachedSizesToArray(data);
+
+    // Done! no read to write data body for client
+    return true;
+}
+} // namespace rpc
+} // namespace Drill

http://git-wip-us.apache.org/repos/asf/drill/blob/166c4ce7/contrib/native/client/src/clientlib/rpcMessage.hpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/clientlib/rpcMessage.hpp b/contrib/native/client/src/clientlib/rpcMessage.hpp
index 6696971..15487e9 100644
--- a/contrib/native/client/src/clientlib/rpcMessage.hpp
+++ b/contrib/native/client/src/clientlib/rpcMessage.hpp
@@ -25,8 +25,8 @@
 #include "GeneralRPC.pb.h"
 
 namespace Drill {
-
-class InBoundRpcMessage {
+namespace rpc {
+struct InBoundRpcMessage {
     public:
         exec::rpc::RpcMode m_mode;
         int m_rpc_type;
@@ -39,7 +39,7 @@ class InBoundRpcMessage {
         bool has_rpc_type() { return m_has_rpc_type; };
 };
 
-class OutBoundRpcMessage {
+struct OutBoundRpcMessage {
     public:
         exec::rpc::RpcMode m_mode;
         int m_rpc_type;
@@ -49,6 +49,13 @@ class OutBoundRpcMessage {
             m_mode(mode), m_rpc_type(rpc_type), m_coord_id(coord_id), m_pbody(pbody) { }
 };
 
-}
+std::size_t lengthDecode(const uint8_t* buf, uint32_t& length);
+
+bool decode(const uint8_t* buf, int length, InBoundRpcMessage& msg);
+
+bool encode(DataBuf& buf, const OutBoundRpcMessage& msg);
+
+} // namespace rpc
+} // namespace Drill
 
 #endif

http://git-wip-us.apache.org/repos/asf/drill/blob/166c4ce7/contrib/native/client/src/clientlib/utils.cpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/clientlib/utils.cpp b/contrib/native/client/src/clientlib/utils.cpp
index 1e6a877..d3c8f08 100644
--- a/contrib/native/client/src/clientlib/utils.cpp
+++ b/contrib/native/client/src/clientlib/utils.cpp
@@ -22,6 +22,13 @@
 #include "logger.hpp"
 #include "drill/common.hpp"
 
+#if defined _WIN32  || defined _WIN64
+//Windows header files redefine 'max'
+#ifdef max
+#undef max
+#endif
+#endif
+
 namespace Drill{
 
 

http://git-wip-us.apache.org/repos/asf/drill/blob/166c4ce7/contrib/native/client/src/clientlib/utils.hpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/clientlib/utils.hpp b/contrib/native/client/src/clientlib/utils.hpp
index 3237aa3..4cd8fa5 100644
--- a/contrib/native/client/src/clientlib/utils.hpp
+++ b/contrib/native/client/src/clientlib/utils.hpp
@@ -31,7 +31,6 @@
     #undef random
   #endif
 #endif
-#include <boost/asio/deadline_timer.hpp>
 #include <boost/random/mersenne_twister.hpp> // for mt19937
 #include <boost/random/random_device.hpp>
 #include <boost/random/uniform_int.hpp>

http://git-wip-us.apache.org/repos/asf/drill/blob/166c4ce7/contrib/native/client/src/clientlib/y2038/time64.c
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/clientlib/y2038/time64.c b/contrib/native/client/src/clientlib/y2038/time64.c
index e0d61c8..bbbabe2 100644
--- a/contrib/native/client/src/clientlib/y2038/time64.c
+++ b/contrib/native/client/src/clientlib/y2038/time64.c
@@ -110,15 +110,15 @@ static const int safe_years_low[SOLAR_CYCLE_LENGTH] = {
 };
 
 /* This isn't used, but it's handy to look at */
-static const char dow_year_start[SOLAR_CYCLE_LENGTH] = {
-    5, 0, 1, 2,     /* 0       2016 - 2019 */
-    3, 5, 6, 0,     /* 4  */
-    1, 3, 4, 5,     /* 8       1996 - 1998, 1971*/
-    6, 1, 2, 3,     /* 12      1972 - 1975 */
-    4, 6, 0, 1,     /* 16 */
-    2, 4, 5, 6,     /* 20      2036, 2037, 2010, 2011 */
-    0, 2, 3, 4      /* 24      2012, 2013, 2014, 2015 */
-};
+//static const char dow_year_start[SOLAR_CYCLE_LENGTH] = {
+//    5, 0, 1, 2,     /* 0       2016 - 2019 */
+//    3, 5, 6, 0,     /* 4  */
+//    1, 3, 4, 5,     /* 8       1996 - 1998, 1971*/
+//    6, 1, 2, 3,     /* 12      1972 - 1975 */
+//    4, 6, 0, 1,     /* 16 */
+//    2, 4, 5, 6,     /* 20      2036, 2037, 2010, 2011 */
+//    0, 2, 3, 4      /* 24      2012, 2013, 2014, 2015 */
+//};
 
 /* Let's assume people are going to be looking for dates in the future.
    Let's provide some cheats so you can skip ahead.

http://git-wip-us.apache.org/repos/asf/drill/blob/166c4ce7/contrib/native/client/src/clientlib/zookeeperClient.cpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/clientlib/zookeeperClient.cpp b/contrib/native/client/src/clientlib/zookeeperClient.cpp
new file mode 100644
index 0000000..535bebc
--- /dev/null
+++ b/contrib/native/client/src/clientlib/zookeeperClient.cpp
@@ -0,0 +1,168 @@
+/*
+ * 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.
+ */
+
+#include <boost/bind.hpp>
+#include <drill/drillClient.hpp>
+#include "zookeeperClient.hpp"
+
+#include "errmsgs.hpp"
+#include "logger.hpp"
+
+namespace Drill {
+std::string ZookeeperClient::s_defaultDrillPath("/drill/drillbits1");
+static void watcherCallback(zhandle_t *zzh, int type, int state, const char *path, void* context) {
+	static_cast<ZookeeperClient*>(context)->watcher(zzh, type, state, path, context);
+}
+
+ZookeeperClient::ZookeeperClient(const std::string& drillPath)
+: p_zh(), m_state(), m_path(drillPath) {
+    m_bConnecting=true;
+    memset(&m_id, 0, sizeof(m_id));
+}
+
+ZookeeperClient::~ZookeeperClient(){
+}
+
+ZooLogLevel ZookeeperClient::getZkLogLevel(){
+    //typedef enum {ZOO_LOG_LEVEL_ERROR=1,
+    //    ZOO_LOG_LEVEL_WARN=2,
+    //    ZOO_LOG_LEVEL_INFO=3,
+    //    ZOO_LOG_LEVEL_DEBUG=4
+    //} ZooLogLevel;
+    switch(DrillClientConfig::getLogLevel()){
+        case LOG_TRACE:
+        case LOG_DEBUG:
+            return ZOO_LOG_LEVEL_DEBUG;
+        case LOG_INFO:
+            return ZOO_LOG_LEVEL_INFO;
+        case LOG_WARNING:
+            return ZOO_LOG_LEVEL_WARN;
+        case LOG_ERROR:
+        case LOG_FATAL:
+        default:
+            return ZOO_LOG_LEVEL_ERROR;
+    }
+    return ZOO_LOG_LEVEL_ERROR;
+}
+
+void ZookeeperClient::watcher(zhandle_t *zzh, int type, int state, const char *path, void*) {
+    //From cli.c
+
+    /* Be careful using zh here rather than zzh - as this may be mt code
+     * the client lib may call the watcher before zookeeper_init returns */
+
+    this->m_state=state;
+    if (type == ZOO_SESSION_EVENT) {
+        if (state == ZOO_CONNECTED_STATE) {
+        } else if (state == ZOO_AUTH_FAILED_STATE) {
+            this->m_err= getMessage(ERR_CONN_ZKNOAUTH);
+            this->close();
+        } else if (state == ZOO_EXPIRED_SESSION_STATE) {
+        	this->m_err= getMessage(ERR_CONN_ZKEXP);
+        	this->close();
+        }
+    }
+    // signal the cond var
+    {
+        if (state == ZOO_CONNECTED_STATE){
+            DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Connected to Zookeeper." << std::endl;)
+        }
+        boost::lock_guard<boost::mutex> bufferLock(this->m_cvMutex);
+        this->m_bConnecting=false;
+    }
+    this->m_cv.notify_one();
+}
+
+int ZookeeperClient::getAllDrillbits(const std::string& connectStr, std::vector<std::string>& drillbits){
+    uint32_t waitTime=30000; // 10 seconds
+    zoo_set_debug_level(getZkLogLevel());
+    zoo_deterministic_conn_order(1); // enable deterministic order
+
+    p_zh = boost::shared_ptr<zhandle_t>(zookeeper_init(connectStr.c_str(), &watcherCallback, waitTime, &m_id, this, 0), zookeeper_close);
+    if(!p_zh) {
+        m_err = getMessage(ERR_CONN_ZKFAIL);
+        return -1;
+    }
+
+    m_err="";
+	//Wait for the completion handler to signal successful connection
+	boost::unique_lock<boost::mutex> bufferLock(this->m_cvMutex);
+	boost::system_time const timeout=boost::get_system_time()+ boost::posix_time::milliseconds(waitTime);
+	while(this->m_bConnecting) {
+		if(!this->m_cv.timed_wait(bufferLock, timeout)){
+			m_err = getMessage(ERR_CONN_ZKTIMOUT);
+			return -1;
+		}
+	}
+
+    if(m_state!=ZOO_CONNECTED_STATE){
+        return -1;
+    }
+
+    int rc = ZOK;
+
+    struct String_vector drillbitsVector;
+    rc=zoo_get_children(p_zh.get(), m_path.c_str(), 0, &drillbitsVector);
+    if(rc!=ZOK){
+        m_err=getMessage(ERR_CONN_ZKERR, rc);
+        p_zh.reset();
+        return -1;
+    }
+
+    // Make sure we deallocate drillbitsVector properly when we exit
+    boost::shared_ptr<String_vector> guard(&drillbitsVector, deallocate_String_vector);
+
+    if(drillbitsVector.count > 0){
+        DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "Found " << drillbitsVector.count << " drillbits in cluster ("
+                << connectStr << "/" << m_path
+                << ")." <<std::endl;)
+		for(int i=0; i<drillbitsVector.count; i++){
+			drillbits.push_back(drillbitsVector.data[i]);
+		}
+        for(int i=0; i<drillbits.size(); i++){
+            DRILL_MT_LOG(DRILL_LOG(LOG_TRACE) << "\t Unshuffled Drillbit id: " << drillbits[i] << std::endl;)
+        }
+    }
+    return 0;
+}
+
+int ZookeeperClient::getEndPoint(const std::string& drillbit, exec::DrillbitEndpoint& endpoint){
+    int rc = ZOK;
+	// pick the drillbit at 'index'
+	std::string s(m_path +  "/" + drillbit);
+	int buffer_len=MAX_CONNECT_STR;
+	char buffer[MAX_CONNECT_STR+1];
+	struct Stat stat;
+	buffer[MAX_CONNECT_STR]=0;
+	rc= zoo_get(p_zh.get(), s.c_str(), 0, buffer,  &buffer_len, &stat);
+	if(rc!=ZOK){
+		m_err=getMessage(ERR_CONN_ZKDBITERR, rc);
+		return -1;
+	}
+	exec::DrillServiceInstance drillServiceInstance;
+	drillServiceInstance.ParseFromArray(buffer, buffer_len);
+	endpoint=drillServiceInstance.endpoint();
+
+    return 0;
+}
+
+void ZookeeperClient::close(){
+	p_zh.reset();
+}
+
+} /* namespace Drill */

http://git-wip-us.apache.org/repos/asf/drill/blob/166c4ce7/contrib/native/client/src/clientlib/zookeeperClient.hpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/clientlib/zookeeperClient.hpp b/contrib/native/client/src/clientlib/zookeeperClient.hpp
new file mode 100644
index 0000000..25d6af5
--- /dev/null
+++ b/contrib/native/client/src/clientlib/zookeeperClient.hpp
@@ -0,0 +1,71 @@
+/*
+ * 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.
+ */
+
+#ifdef _WIN32
+#include <zookeeper.h>
+#else
+#include <zookeeper/zookeeper.h>
+#endif
+
+#include <boost/shared_ptr.hpp>
+#include <boost/thread/condition_variable.hpp>
+#include <boost/thread/mutex.hpp>
+
+#include "UserBitShared.pb.h"
+
+
+#ifndef ZOOKEEPER_CLIENT_H
+#define ZOOKEEPER_CLIENT_H
+
+namespace Drill {
+class ZookeeperClient{
+    public:
+		static std::string s_defaultDrillPath;
+
+        ZookeeperClient(const std::string& drillPath = s_defaultDrillPath);
+        ~ZookeeperClient();
+        static ZooLogLevel getZkLogLevel();
+        // comma separated host:port pairs, each corresponding to a zk
+        // server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002
+        void close();
+        const std::string& getError() const{return m_err;}
+        // return unshuffled list of drillbits
+        int getAllDrillbits(const std::string& connectStr, std::vector<std::string>& drillbits);
+        // picks the index drillbit and returns the corresponding endpoint object
+        int getEndPoint(const std::string& drillbit, exec::DrillbitEndpoint& endpoint);
+
+        void watcher(zhandle_t *zzh, int type, int state, const char *path, void* context);
+
+    private:
+        boost::shared_ptr<zhandle_t> p_zh;
+        clientid_t m_id;
+        int m_state;
+        std::string m_err;
+
+        boost::mutex m_cvMutex;
+        // Condition variable to signal connection callback has been processed
+        boost::condition_variable m_cv;
+        bool m_bConnecting;
+        std::string m_path;
+
+};
+} /* namespace Drill */
+
+
+
+#endif /* ZOOKEEPER_H */

http://git-wip-us.apache.org/repos/asf/drill/blob/166c4ce7/contrib/native/client/src/include/drill/collections.hpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/include/drill/collections.hpp b/contrib/native/client/src/include/drill/collections.hpp
new file mode 100644
index 0000000..9fbfcc5
--- /dev/null
+++ b/contrib/native/client/src/include/drill/collections.hpp
@@ -0,0 +1,179 @@
+/*
+ * 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.
+ */
+
+#ifndef _DRILL_COLLECTIONS_H
+#define _DRILL_COLLECTIONS_H
+
+#include <iterator>
+
+#include <boost/noncopyable.hpp>
+#include <boost/shared_ptr.hpp>
+
+namespace Drill {
+namespace impl {
+
+/**
+ * Interface for internal iterators
+ */
+template<typename T>
+class DrillIteratorImpl: private boost::noncopyable {
+public:
+	typedef DrillIteratorImpl<T> iterator;
+	typedef boost::shared_ptr<iterator> iterator_ptr;
+
+	typedef T value_type;
+	typedef value_type& reference;
+	typedef value_type* pointer;
+
+	virtual ~DrillIteratorImpl() {};
+
+	// To allow conversion from non-const to const types
+	virtual operator typename DrillIteratorImpl<const T>::iterator_ptr() const = 0;
+
+	virtual reference operator*() const = 0;
+	virtual pointer   operator->() const = 0;
+
+	virtual iterator& operator++() = 0;
+
+	virtual bool operator==(const iterator& x) const = 0;
+	virtual bool operator!=(const iterator& x) const = 0;
+};
+
+/**
+ * Interface for internal collections
+ */
+template<typename T>
+class DrillCollectionImpl: private boost::noncopyable {
+public:
+	// STL-like iterator typedef
+	typedef DrillIteratorImpl<T> iterator;
+	typedef boost::shared_ptr<iterator> iterator_ptr;
+	typedef DrillIteratorImpl<const T> const_iterator;
+	typedef boost::shared_ptr<const_iterator> const_iterator_ptr;
+
+	typedef T value_type;
+	typedef value_type& reference;
+	typedef const value_type& const_reference;
+	typedef value_type* pointer;
+	typedef const value_type* const_pointer;
+	typedef int size_type;
+
+	virtual ~DrillCollectionImpl() {}
+
+	virtual iterator_ptr begin() = 0;
+	virtual const_iterator_ptr begin() const = 0;
+	virtual iterator_ptr end() = 0;
+	virtual const_iterator_ptr end() const = 0;
+};
+} // namespace internal
+
+template<typename T>
+class DrillCollection;
+
+template<typename T>
+class DrillIterator: public std::iterator<std::input_iterator_tag, T> {
+public:
+	typedef impl::DrillIteratorImpl<T> Impl;
+	typedef boost::shared_ptr<Impl> ImplPtr;
+
+	typedef DrillIterator<T> iterator;
+	typedef std::iterator<std::input_iterator_tag, T> superclass;
+	typedef typename superclass::reference reference;
+	typedef typename superclass::pointer pointer;
+
+	// Default constructor
+	DrillIterator(): m_pImpl() {};
+	~DrillIterator() {}
+
+	// Iterators are CopyConstructible and CopyAssignable
+	DrillIterator(const iterator& it): m_pImpl(it.m_pImpl) {}
+	iterator& operator=(const iterator& it) {
+		m_pImpl = it.m_pImpl;
+		return *this;
+	}
+
+	template<typename U>
+	DrillIterator(const DrillIterator<U>& it): m_pImpl(*it.m_pImpl) {}
+
+	reference operator*() const { return m_pImpl->operator*(); }
+	pointer   operator->() const { return m_pImpl->operator->(); }
+
+	iterator& operator++() { m_pImpl->operator++(); return *this; }
+
+	bool operator==(const iterator& x) const { 
+		if (m_pImpl == x.m_pImpl) {
+			return true;
+		}
+		return m_pImpl && m_pImpl->operator==(*x.m_pImpl);
+	}
+
+	bool operator!=(const iterator& x) const { 
+		if (m_pImpl == x.m_pImpl) {
+			return false;
+		}
+		return !m_pImpl ||  m_pImpl->operator!=(*x.m_pImpl);
+	}
+
+private:
+	template<typename U>
+	friend class DrillCollection;
+	template<typename U>
+	friend class DrillIterator;
+
+	ImplPtr m_pImpl;
+
+	template<typename U>
+	DrillIterator(const boost::shared_ptr<impl::DrillIteratorImpl<U> >& pImpl): m_pImpl(pImpl) {}
+};
+
+template<typename T>
+class DrillCollection {
+public:
+	typedef impl::DrillCollectionImpl<T> Impl;
+	typedef boost::shared_ptr<Impl> ImplPtr;
+
+	// STL-like iterator typedef
+	typedef DrillIterator<T> iterator;
+	typedef DrillIterator<const T> const_iterator;
+	typedef T value_type;
+	typedef value_type& reference;
+	typedef const value_type& const_reference;
+	typedef value_type* pointer;
+	typedef const value_type* const_pointer;
+	typedef int size_type;
+
+	iterator       begin()       { return iterator(m_pImpl->begin()); }
+	const_iterator begin() const { return const_iterator(boost::const_pointer_cast<const Impl>(m_pImpl)->begin()); }
+	iterator       end()         { return iterator(m_pImpl->end()); }
+	const_iterator end() const   { return const_iterator(boost::const_pointer_cast<const Impl>(m_pImpl)->end()); }
+
+protected:
+	DrillCollection(const ImplPtr& impl): m_pImpl(impl) {}
+
+	Impl& operator*() { return *m_pImpl; }
+	const Impl& operator*() const { return *m_pImpl; }
+	Impl* operator->() { return m_pImpl.get(); }
+	const Impl* operator->() const { return m_pImpl.get(); }
+
+private:
+	ImplPtr m_pImpl;
+};
+
+
+} /* namespace Drill */
+#endif /* _DRILL_COLLECTIONS_H */

http://git-wip-us.apache.org/repos/asf/drill/blob/166c4ce7/contrib/native/client/src/include/drill/common.hpp
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/include/drill/common.hpp b/contrib/native/client/src/include/drill/common.hpp
index a617dc7..6d3816e 100644
--- a/contrib/native/client/src/include/drill/common.hpp
+++ b/contrib/native/client/src/include/drill/common.hpp
@@ -20,6 +20,24 @@
 #ifndef _COMMON_H_
 #define _COMMON_H_
 
+#if defined _WIN32 || defined __CYGWIN__
+  #ifdef DRILL_CLIENT_EXPORTS
+      #define DECLSPEC_DRILL_CLIENT __declspec(dllexport)
+  #else
+    #ifdef USE_STATIC_LIBDRILL
+      #define DECLSPEC_DRILL_CLIENT
+    #else
+      #define DECLSPEC_DRILL_CLIENT  __declspec(dllimport)
+    #endif
+  #endif
+#else
+  #if __GNUC__ >= 4
+    #define DECLSPEC_DRILL_CLIENT __attribute__ ((visibility ("default")))
+  #else
+    #define DECLSPEC_DRILL_CLIENT
+  #endif
+#endif
+
 #ifdef _WIN32
 // The order of inclusion is important. Including winsock2 before everything else
 // ensures that the correct typedefs are defined and that the older typedefs defined


[06/15] drill git commit: DRILL-4853: Update C++ protobuf source files

Posted by pa...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/2558803e/contrib/native/client/src/protobuf/UserBitShared.pb.cc
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/protobuf/UserBitShared.pb.cc b/contrib/native/client/src/protobuf/UserBitShared.pb.cc
index c314a1d..abd6b4e 100644
--- a/contrib/native/client/src/protobuf/UserBitShared.pb.cc
+++ b/contrib/native/client/src/protobuf/UserBitShared.pb.cc
@@ -81,6 +81,12 @@ const ::google::protobuf::internal::GeneratedMessageReflection*
 const ::google::protobuf::Descriptor* MetricValue_descriptor_ = NULL;
 const ::google::protobuf::internal::GeneratedMessageReflection*
   MetricValue_reflection_ = NULL;
+const ::google::protobuf::Descriptor* Registry_descriptor_ = NULL;
+const ::google::protobuf::internal::GeneratedMessageReflection*
+  Registry_reflection_ = NULL;
+const ::google::protobuf::Descriptor* Jar_descriptor_ = NULL;
+const ::google::protobuf::internal::GeneratedMessageReflection*
+  Jar_reflection_ = NULL;
 const ::google::protobuf::EnumDescriptor* RpcChannel_descriptor_ = NULL;
 const ::google::protobuf::EnumDescriptor* QueryType_descriptor_ = NULL;
 const ::google::protobuf::EnumDescriptor* FragmentState_descriptor_ = NULL;
@@ -454,6 +460,37 @@ void protobuf_AssignDesc_UserBitShared_2eproto() {
       ::google::protobuf::DescriptorPool::generated_pool(),
       ::google::protobuf::MessageFactory::generated_factory(),
       sizeof(MetricValue));
+  Registry_descriptor_ = file->message_type(19);
+  static const int Registry_offsets_[1] = {
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(Registry, jar_),
+  };
+  Registry_reflection_ =
+    new ::google::protobuf::internal::GeneratedMessageReflection(
+      Registry_descriptor_,
+      Registry::default_instance_,
+      Registry_offsets_,
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(Registry, _has_bits_[0]),
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(Registry, _unknown_fields_),
+      -1,
+      ::google::protobuf::DescriptorPool::generated_pool(),
+      ::google::protobuf::MessageFactory::generated_factory(),
+      sizeof(Registry));
+  Jar_descriptor_ = file->message_type(20);
+  static const int Jar_offsets_[2] = {
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(Jar, name_),
+    GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(Jar, function_signature_),
+  };
+  Jar_reflection_ =
+    new ::google::protobuf::internal::GeneratedMessageReflection(
+      Jar_descriptor_,
+      Jar::default_instance_,
+      Jar_offsets_,
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(Jar, _has_bits_[0]),
+      GOOGLE_PROTOBUF_GENERATED_MESSAGE_FIELD_OFFSET(Jar, _unknown_fields_),
+      -1,
+      ::google::protobuf::DescriptorPool::generated_pool(),
+      ::google::protobuf::MessageFactory::generated_factory(),
+      sizeof(Jar));
   RpcChannel_descriptor_ = file->enum_type(0);
   QueryType_descriptor_ = file->enum_type(1);
   FragmentState_descriptor_ = file->enum_type(2);
@@ -508,6 +545,10 @@ void protobuf_RegisterTypes(const ::std::string&) {
     StreamProfile_descriptor_, &StreamProfile::default_instance());
   ::google::protobuf::MessageFactory::InternalRegisterGeneratedMessage(
     MetricValue_descriptor_, &MetricValue::default_instance());
+  ::google::protobuf::MessageFactory::InternalRegisterGeneratedMessage(
+    Registry_descriptor_, &Registry::default_instance());
+  ::google::protobuf::MessageFactory::InternalRegisterGeneratedMessage(
+    Jar_descriptor_, &Jar::default_instance());
 }
 
 }  // namespace
@@ -553,6 +594,10 @@ void protobuf_ShutdownFile_UserBitShared_2eproto() {
   delete StreamProfile_reflection_;
   delete MetricValue::default_instance_;
   delete MetricValue_reflection_;
+  delete Registry::default_instance_;
+  delete Registry_reflection_;
+  delete Jar::default_instance_;
+  delete Jar_reflection_;
 }
 
 void protobuf_AddDesc_UserBitShared_2eproto() {
@@ -650,32 +695,36 @@ void protobuf_AddDesc_UserBitShared_2eproto() {
     "\017\n\007records\030\001 \001(\003\022\017\n\007batches\030\002 \001(\003\022\017\n\007sch"
     "emas\030\003 \001(\003\"J\n\013MetricValue\022\021\n\tmetric_id\030\001"
     " \001(\005\022\022\n\nlong_value\030\002 \001(\003\022\024\n\014double_value"
-    "\030\003 \001(\001*5\n\nRpcChannel\022\017\n\013BIT_CONTROL\020\000\022\014\n"
-    "\010BIT_DATA\020\001\022\010\n\004USER\020\002*/\n\tQueryType\022\007\n\003SQ"
-    "L\020\001\022\013\n\007LOGICAL\020\002\022\014\n\010PHYSICAL\020\003*\207\001\n\rFragm"
-    "entState\022\013\n\007SENDING\020\000\022\027\n\023AWAITING_ALLOCA"
-    "TION\020\001\022\013\n\007RUNNING\020\002\022\014\n\010FINISHED\020\003\022\r\n\tCAN"
-    "CELLED\020\004\022\n\n\006FAILED\020\005\022\032\n\026CANCELLATION_REQ"
-    "UESTED\020\006*\335\005\n\020CoreOperatorType\022\021\n\rSINGLE_"
-    "SENDER\020\000\022\024\n\020BROADCAST_SENDER\020\001\022\n\n\006FILTER"
-    "\020\002\022\022\n\016HASH_AGGREGATE\020\003\022\r\n\tHASH_JOIN\020\004\022\016\n"
-    "\nMERGE_JOIN\020\005\022\031\n\025HASH_PARTITION_SENDER\020\006"
-    "\022\t\n\005LIMIT\020\007\022\024\n\020MERGING_RECEIVER\020\010\022\034\n\030ORD"
-    "ERED_PARTITION_SENDER\020\t\022\013\n\007PROJECT\020\n\022\026\n\022"
-    "UNORDERED_RECEIVER\020\013\022\020\n\014RANGE_SENDER\020\014\022\n"
-    "\n\006SCREEN\020\r\022\034\n\030SELECTION_VECTOR_REMOVER\020\016"
-    "\022\027\n\023STREAMING_AGGREGATE\020\017\022\016\n\nTOP_N_SORT\020"
-    "\020\022\021\n\rEXTERNAL_SORT\020\021\022\t\n\005TRACE\020\022\022\t\n\005UNION"
-    "\020\023\022\014\n\010OLD_SORT\020\024\022\032\n\026PARQUET_ROW_GROUP_SC"
-    "AN\020\025\022\021\n\rHIVE_SUB_SCAN\020\026\022\025\n\021SYSTEM_TABLE_"
-    "SCAN\020\027\022\021\n\rMOCK_SUB_SCAN\020\030\022\022\n\016PARQUET_WRI"
-    "TER\020\031\022\023\n\017DIRECT_SUB_SCAN\020\032\022\017\n\013TEXT_WRITE"
-    "R\020\033\022\021\n\rTEXT_SUB_SCAN\020\034\022\021\n\rJSON_SUB_SCAN\020"
-    "\035\022\030\n\024INFO_SCHEMA_SUB_SCAN\020\036\022\023\n\017COMPLEX_T"
-    "O_JSON\020\037\022\025\n\021PRODUCER_CONSUMER\020 \022\022\n\016HBASE"
-    "_SUB_SCAN\020!\022\n\n\006WINDOW\020\"\022\024\n\020NESTED_LOOP_J"
-    "OIN\020#\022\021\n\rAVRO_SUB_SCAN\020$B.\n\033org.apache.d"
-    "rill.exec.protoB\rUserBitSharedH\001", 4432);
+    "\030\003 \001(\001\")\n\010Registry\022\035\n\003jar\030\001 \003(\0132\020.exec.s"
+    "hared.Jar\"/\n\003Jar\022\014\n\004name\030\001 \001(\t\022\032\n\022functi"
+    "on_signature\030\002 \003(\t*5\n\nRpcChannel\022\017\n\013BIT_"
+    "CONTROL\020\000\022\014\n\010BIT_DATA\020\001\022\010\n\004USER\020\002*V\n\tQue"
+    "ryType\022\007\n\003SQL\020\001\022\013\n\007LOGICAL\020\002\022\014\n\010PHYSICAL"
+    "\020\003\022\r\n\tEXECUTION\020\004\022\026\n\022PREPARED_STATEMENT\020"
+    "\005*\207\001\n\rFragmentState\022\013\n\007SENDING\020\000\022\027\n\023AWAI"
+    "TING_ALLOCATION\020\001\022\013\n\007RUNNING\020\002\022\014\n\010FINISH"
+    "ED\020\003\022\r\n\tCANCELLED\020\004\022\n\n\006FAILED\020\005\022\032\n\026CANCE"
+    "LLATION_REQUESTED\020\006*\335\005\n\020CoreOperatorType"
+    "\022\021\n\rSINGLE_SENDER\020\000\022\024\n\020BROADCAST_SENDER\020"
+    "\001\022\n\n\006FILTER\020\002\022\022\n\016HASH_AGGREGATE\020\003\022\r\n\tHAS"
+    "H_JOIN\020\004\022\016\n\nMERGE_JOIN\020\005\022\031\n\025HASH_PARTITI"
+    "ON_SENDER\020\006\022\t\n\005LIMIT\020\007\022\024\n\020MERGING_RECEIV"
+    "ER\020\010\022\034\n\030ORDERED_PARTITION_SENDER\020\t\022\013\n\007PR"
+    "OJECT\020\n\022\026\n\022UNORDERED_RECEIVER\020\013\022\020\n\014RANGE"
+    "_SENDER\020\014\022\n\n\006SCREEN\020\r\022\034\n\030SELECTION_VECTO"
+    "R_REMOVER\020\016\022\027\n\023STREAMING_AGGREGATE\020\017\022\016\n\n"
+    "TOP_N_SORT\020\020\022\021\n\rEXTERNAL_SORT\020\021\022\t\n\005TRACE"
+    "\020\022\022\t\n\005UNION\020\023\022\014\n\010OLD_SORT\020\024\022\032\n\026PARQUET_R"
+    "OW_GROUP_SCAN\020\025\022\021\n\rHIVE_SUB_SCAN\020\026\022\025\n\021SY"
+    "STEM_TABLE_SCAN\020\027\022\021\n\rMOCK_SUB_SCAN\020\030\022\022\n\016"
+    "PARQUET_WRITER\020\031\022\023\n\017DIRECT_SUB_SCAN\020\032\022\017\n"
+    "\013TEXT_WRITER\020\033\022\021\n\rTEXT_SUB_SCAN\020\034\022\021\n\rJSO"
+    "N_SUB_SCAN\020\035\022\030\n\024INFO_SCHEMA_SUB_SCAN\020\036\022\023"
+    "\n\017COMPLEX_TO_JSON\020\037\022\025\n\021PRODUCER_CONSUMER"
+    "\020 \022\022\n\016HBASE_SUB_SCAN\020!\022\n\n\006WINDOW\020\"\022\024\n\020NE"
+    "STED_LOOP_JOIN\020#\022\021\n\rAVRO_SUB_SCAN\020$B.\n\033o"
+    "rg.apache.drill.exec.protoB\rUserBitShare"
+    "dH\001", 4563);
   ::google::protobuf::MessageFactory::InternalRegisterGeneratedFile(
     "UserBitShared.proto", &protobuf_RegisterTypes);
   UserCredentials::default_instance_ = new UserCredentials();
@@ -701,6 +750,8 @@ void protobuf_AddDesc_UserBitShared_2eproto() {
   OperatorProfile::default_instance_ = new OperatorProfile();
   StreamProfile::default_instance_ = new StreamProfile();
   MetricValue::default_instance_ = new MetricValue();
+  Registry::default_instance_ = new Registry();
+  Jar::default_instance_ = new Jar();
   UserCredentials::default_instance_->InitAsDefaultInstance();
   QueryId::default_instance_->InitAsDefaultInstance();
   DrillPBError::default_instance_->InitAsDefaultInstance();
@@ -720,6 +771,8 @@ void protobuf_AddDesc_UserBitShared_2eproto() {
   OperatorProfile::default_instance_->InitAsDefaultInstance();
   StreamProfile::default_instance_->InitAsDefaultInstance();
   MetricValue::default_instance_->InitAsDefaultInstance();
+  Registry::default_instance_->InitAsDefaultInstance();
+  Jar::default_instance_->InitAsDefaultInstance();
   ::google::protobuf::internal::OnShutdown(&protobuf_ShutdownFile_UserBitShared_2eproto);
 }
 
@@ -753,6 +806,8 @@ bool QueryType_IsValid(int value) {
     case 1:
     case 2:
     case 3:
+    case 4:
+    case 5:
       return true;
     default:
       return false;
@@ -8134,6 +8189,482 @@ void MetricValue::Swap(MetricValue* other) {
 }
 
 
+// ===================================================================
+
+#ifndef _MSC_VER
+const int Registry::kJarFieldNumber;
+#endif  // !_MSC_VER
+
+Registry::Registry()
+  : ::google::protobuf::Message() {
+  SharedCtor();
+}
+
+void Registry::InitAsDefaultInstance() {
+}
+
+Registry::Registry(const Registry& from)
+  : ::google::protobuf::Message() {
+  SharedCtor();
+  MergeFrom(from);
+}
+
+void Registry::SharedCtor() {
+  _cached_size_ = 0;
+  ::memset(_has_bits_, 0, sizeof(_has_bits_));
+}
+
+Registry::~Registry() {
+  SharedDtor();
+}
+
+void Registry::SharedDtor() {
+  if (this != default_instance_) {
+  }
+}
+
+void Registry::SetCachedSize(int size) const {
+  GOOGLE_SAFE_CONCURRENT_WRITES_BEGIN();
+  _cached_size_ = size;
+  GOOGLE_SAFE_CONCURRENT_WRITES_END();
+}
+const ::google::protobuf::Descriptor* Registry::descriptor() {
+  protobuf_AssignDescriptorsOnce();
+  return Registry_descriptor_;
+}
+
+const Registry& Registry::default_instance() {
+  if (default_instance_ == NULL) protobuf_AddDesc_UserBitShared_2eproto();
+  return *default_instance_;
+}
+
+Registry* Registry::default_instance_ = NULL;
+
+Registry* Registry::New() const {
+  return new Registry;
+}
+
+void Registry::Clear() {
+  jar_.Clear();
+  ::memset(_has_bits_, 0, sizeof(_has_bits_));
+  mutable_unknown_fields()->Clear();
+}
+
+bool Registry::MergePartialFromCodedStream(
+    ::google::protobuf::io::CodedInputStream* input) {
+#define DO_(EXPRESSION) if (!(EXPRESSION)) return false
+  ::google::protobuf::uint32 tag;
+  while ((tag = input->ReadTag()) != 0) {
+    switch (::google::protobuf::internal::WireFormatLite::GetTagFieldNumber(tag)) {
+      // repeated .exec.shared.Jar jar = 1;
+      case 1: {
+        if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
+            ::google::protobuf::internal::WireFormatLite::WIRETYPE_LENGTH_DELIMITED) {
+         parse_jar:
+          DO_(::google::protobuf::internal::WireFormatLite::ReadMessageNoVirtual(
+                input, add_jar()));
+        } else {
+          goto handle_uninterpreted;
+        }
+        if (input->ExpectTag(10)) goto parse_jar;
+        if (input->ExpectAtEnd()) return true;
+        break;
+      }
+
+      default: {
+      handle_uninterpreted:
+        if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
+            ::google::protobuf::internal::WireFormatLite::WIRETYPE_END_GROUP) {
+          return true;
+        }
+        DO_(::google::protobuf::internal::WireFormat::SkipField(
+              input, tag, mutable_unknown_fields()));
+        break;
+      }
+    }
+  }
+  return true;
+#undef DO_
+}
+
+void Registry::SerializeWithCachedSizes(
+    ::google::protobuf::io::CodedOutputStream* output) const {
+  // repeated .exec.shared.Jar jar = 1;
+  for (int i = 0; i < this->jar_size(); i++) {
+    ::google::protobuf::internal::WireFormatLite::WriteMessageMaybeToArray(
+      1, this->jar(i), output);
+  }
+
+  if (!unknown_fields().empty()) {
+    ::google::protobuf::internal::WireFormat::SerializeUnknownFields(
+        unknown_fields(), output);
+  }
+}
+
+::google::protobuf::uint8* Registry::SerializeWithCachedSizesToArray(
+    ::google::protobuf::uint8* target) const {
+  // repeated .exec.shared.Jar jar = 1;
+  for (int i = 0; i < this->jar_size(); i++) {
+    target = ::google::protobuf::internal::WireFormatLite::
+      WriteMessageNoVirtualToArray(
+        1, this->jar(i), target);
+  }
+
+  if (!unknown_fields().empty()) {
+    target = ::google::protobuf::internal::WireFormat::SerializeUnknownFieldsToArray(
+        unknown_fields(), target);
+  }
+  return target;
+}
+
+int Registry::ByteSize() const {
+  int total_size = 0;
+
+  // repeated .exec.shared.Jar jar = 1;
+  total_size += 1 * this->jar_size();
+  for (int i = 0; i < this->jar_size(); i++) {
+    total_size +=
+      ::google::protobuf::internal::WireFormatLite::MessageSizeNoVirtual(
+        this->jar(i));
+  }
+
+  if (!unknown_fields().empty()) {
+    total_size +=
+      ::google::protobuf::internal::WireFormat::ComputeUnknownFieldsSize(
+        unknown_fields());
+  }
+  GOOGLE_SAFE_CONCURRENT_WRITES_BEGIN();
+  _cached_size_ = total_size;
+  GOOGLE_SAFE_CONCURRENT_WRITES_END();
+  return total_size;
+}
+
+void Registry::MergeFrom(const ::google::protobuf::Message& from) {
+  GOOGLE_CHECK_NE(&from, this);
+  const Registry* source =
+    ::google::protobuf::internal::dynamic_cast_if_available<const Registry*>(
+      &from);
+  if (source == NULL) {
+    ::google::protobuf::internal::ReflectionOps::Merge(from, this);
+  } else {
+    MergeFrom(*source);
+  }
+}
+
+void Registry::MergeFrom(const Registry& from) {
+  GOOGLE_CHECK_NE(&from, this);
+  jar_.MergeFrom(from.jar_);
+  mutable_unknown_fields()->MergeFrom(from.unknown_fields());
+}
+
+void Registry::CopyFrom(const ::google::protobuf::Message& from) {
+  if (&from == this) return;
+  Clear();
+  MergeFrom(from);
+}
+
+void Registry::CopyFrom(const Registry& from) {
+  if (&from == this) return;
+  Clear();
+  MergeFrom(from);
+}
+
+bool Registry::IsInitialized() const {
+
+  return true;
+}
+
+void Registry::Swap(Registry* other) {
+  if (other != this) {
+    jar_.Swap(&other->jar_);
+    std::swap(_has_bits_[0], other->_has_bits_[0]);
+    _unknown_fields_.Swap(&other->_unknown_fields_);
+    std::swap(_cached_size_, other->_cached_size_);
+  }
+}
+
+::google::protobuf::Metadata Registry::GetMetadata() const {
+  protobuf_AssignDescriptorsOnce();
+  ::google::protobuf::Metadata metadata;
+  metadata.descriptor = Registry_descriptor_;
+  metadata.reflection = Registry_reflection_;
+  return metadata;
+}
+
+
+// ===================================================================
+
+#ifndef _MSC_VER
+const int Jar::kNameFieldNumber;
+const int Jar::kFunctionSignatureFieldNumber;
+#endif  // !_MSC_VER
+
+Jar::Jar()
+  : ::google::protobuf::Message() {
+  SharedCtor();
+}
+
+void Jar::InitAsDefaultInstance() {
+}
+
+Jar::Jar(const Jar& from)
+  : ::google::protobuf::Message() {
+  SharedCtor();
+  MergeFrom(from);
+}
+
+void Jar::SharedCtor() {
+  _cached_size_ = 0;
+  name_ = const_cast< ::std::string*>(&::google::protobuf::internal::kEmptyString);
+  ::memset(_has_bits_, 0, sizeof(_has_bits_));
+}
+
+Jar::~Jar() {
+  SharedDtor();
+}
+
+void Jar::SharedDtor() {
+  if (name_ != &::google::protobuf::internal::kEmptyString) {
+    delete name_;
+  }
+  if (this != default_instance_) {
+  }
+}
+
+void Jar::SetCachedSize(int size) const {
+  GOOGLE_SAFE_CONCURRENT_WRITES_BEGIN();
+  _cached_size_ = size;
+  GOOGLE_SAFE_CONCURRENT_WRITES_END();
+}
+const ::google::protobuf::Descriptor* Jar::descriptor() {
+  protobuf_AssignDescriptorsOnce();
+  return Jar_descriptor_;
+}
+
+const Jar& Jar::default_instance() {
+  if (default_instance_ == NULL) protobuf_AddDesc_UserBitShared_2eproto();
+  return *default_instance_;
+}
+
+Jar* Jar::default_instance_ = NULL;
+
+Jar* Jar::New() const {
+  return new Jar;
+}
+
+void Jar::Clear() {
+  if (_has_bits_[0 / 32] & (0xffu << (0 % 32))) {
+    if (has_name()) {
+      if (name_ != &::google::protobuf::internal::kEmptyString) {
+        name_->clear();
+      }
+    }
+  }
+  function_signature_.Clear();
+  ::memset(_has_bits_, 0, sizeof(_has_bits_));
+  mutable_unknown_fields()->Clear();
+}
+
+bool Jar::MergePartialFromCodedStream(
+    ::google::protobuf::io::CodedInputStream* input) {
+#define DO_(EXPRESSION) if (!(EXPRESSION)) return false
+  ::google::protobuf::uint32 tag;
+  while ((tag = input->ReadTag()) != 0) {
+    switch (::google::protobuf::internal::WireFormatLite::GetTagFieldNumber(tag)) {
+      // optional string name = 1;
+      case 1: {
+        if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
+            ::google::protobuf::internal::WireFormatLite::WIRETYPE_LENGTH_DELIMITED) {
+          DO_(::google::protobuf::internal::WireFormatLite::ReadString(
+                input, this->mutable_name()));
+          ::google::protobuf::internal::WireFormat::VerifyUTF8String(
+            this->name().data(), this->name().length(),
+            ::google::protobuf::internal::WireFormat::PARSE);
+        } else {
+          goto handle_uninterpreted;
+        }
+        if (input->ExpectTag(18)) goto parse_function_signature;
+        break;
+      }
+
+      // repeated string function_signature = 2;
+      case 2: {
+        if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
+            ::google::protobuf::internal::WireFormatLite::WIRETYPE_LENGTH_DELIMITED) {
+         parse_function_signature:
+          DO_(::google::protobuf::internal::WireFormatLite::ReadString(
+                input, this->add_function_signature()));
+          ::google::protobuf::internal::WireFormat::VerifyUTF8String(
+            this->function_signature(this->function_signature_size() - 1).data(),
+            this->function_signature(this->function_signature_size() - 1).length(),
+            ::google::protobuf::internal::WireFormat::PARSE);
+        } else {
+          goto handle_uninterpreted;
+        }
+        if (input->ExpectTag(18)) goto parse_function_signature;
+        if (input->ExpectAtEnd()) return true;
+        break;
+      }
+
+      default: {
+      handle_uninterpreted:
+        if (::google::protobuf::internal::WireFormatLite::GetTagWireType(tag) ==
+            ::google::protobuf::internal::WireFormatLite::WIRETYPE_END_GROUP) {
+          return true;
+        }
+        DO_(::google::protobuf::internal::WireFormat::SkipField(
+              input, tag, mutable_unknown_fields()));
+        break;
+      }
+    }
+  }
+  return true;
+#undef DO_
+}
+
+void Jar::SerializeWithCachedSizes(
+    ::google::protobuf::io::CodedOutputStream* output) const {
+  // optional string name = 1;
+  if (has_name()) {
+    ::google::protobuf::internal::WireFormat::VerifyUTF8String(
+      this->name().data(), this->name().length(),
+      ::google::protobuf::internal::WireFormat::SERIALIZE);
+    ::google::protobuf::internal::WireFormatLite::WriteString(
+      1, this->name(), output);
+  }
+
+  // repeated string function_signature = 2;
+  for (int i = 0; i < this->function_signature_size(); i++) {
+  ::google::protobuf::internal::WireFormat::VerifyUTF8String(
+    this->function_signature(i).data(), this->function_signature(i).length(),
+    ::google::protobuf::internal::WireFormat::SERIALIZE);
+    ::google::protobuf::internal::WireFormatLite::WriteString(
+      2, this->function_signature(i), output);
+  }
+
+  if (!unknown_fields().empty()) {
+    ::google::protobuf::internal::WireFormat::SerializeUnknownFields(
+        unknown_fields(), output);
+  }
+}
+
+::google::protobuf::uint8* Jar::SerializeWithCachedSizesToArray(
+    ::google::protobuf::uint8* target) const {
+  // optional string name = 1;
+  if (has_name()) {
+    ::google::protobuf::internal::WireFormat::VerifyUTF8String(
+      this->name().data(), this->name().length(),
+      ::google::protobuf::internal::WireFormat::SERIALIZE);
+    target =
+      ::google::protobuf::internal::WireFormatLite::WriteStringToArray(
+        1, this->name(), target);
+  }
+
+  // repeated string function_signature = 2;
+  for (int i = 0; i < this->function_signature_size(); i++) {
+    ::google::protobuf::internal::WireFormat::VerifyUTF8String(
+      this->function_signature(i).data(), this->function_signature(i).length(),
+      ::google::protobuf::internal::WireFormat::SERIALIZE);
+    target = ::google::protobuf::internal::WireFormatLite::
+      WriteStringToArray(2, this->function_signature(i), target);
+  }
+
+  if (!unknown_fields().empty()) {
+    target = ::google::protobuf::internal::WireFormat::SerializeUnknownFieldsToArray(
+        unknown_fields(), target);
+  }
+  return target;
+}
+
+int Jar::ByteSize() const {
+  int total_size = 0;
+
+  if (_has_bits_[0 / 32] & (0xffu << (0 % 32))) {
+    // optional string name = 1;
+    if (has_name()) {
+      total_size += 1 +
+        ::google::protobuf::internal::WireFormatLite::StringSize(
+          this->name());
+    }
+
+  }
+  // repeated string function_signature = 2;
+  total_size += 1 * this->function_signature_size();
+  for (int i = 0; i < this->function_signature_size(); i++) {
+    total_size += ::google::protobuf::internal::WireFormatLite::StringSize(
+      this->function_signature(i));
+  }
+
+  if (!unknown_fields().empty()) {
+    total_size +=
+      ::google::protobuf::internal::WireFormat::ComputeUnknownFieldsSize(
+        unknown_fields());
+  }
+  GOOGLE_SAFE_CONCURRENT_WRITES_BEGIN();
+  _cached_size_ = total_size;
+  GOOGLE_SAFE_CONCURRENT_WRITES_END();
+  return total_size;
+}
+
+void Jar::MergeFrom(const ::google::protobuf::Message& from) {
+  GOOGLE_CHECK_NE(&from, this);
+  const Jar* source =
+    ::google::protobuf::internal::dynamic_cast_if_available<const Jar*>(
+      &from);
+  if (source == NULL) {
+    ::google::protobuf::internal::ReflectionOps::Merge(from, this);
+  } else {
+    MergeFrom(*source);
+  }
+}
+
+void Jar::MergeFrom(const Jar& from) {
+  GOOGLE_CHECK_NE(&from, this);
+  function_signature_.MergeFrom(from.function_signature_);
+  if (from._has_bits_[0 / 32] & (0xffu << (0 % 32))) {
+    if (from.has_name()) {
+      set_name(from.name());
+    }
+  }
+  mutable_unknown_fields()->MergeFrom(from.unknown_fields());
+}
+
+void Jar::CopyFrom(const ::google::protobuf::Message& from) {
+  if (&from == this) return;
+  Clear();
+  MergeFrom(from);
+}
+
+void Jar::CopyFrom(const Jar& from) {
+  if (&from == this) return;
+  Clear();
+  MergeFrom(from);
+}
+
+bool Jar::IsInitialized() const {
+
+  return true;
+}
+
+void Jar::Swap(Jar* other) {
+  if (other != this) {
+    std::swap(name_, other->name_);
+    function_signature_.Swap(&other->function_signature_);
+    std::swap(_has_bits_[0], other->_has_bits_[0]);
+    _unknown_fields_.Swap(&other->_unknown_fields_);
+    std::swap(_cached_size_, other->_cached_size_);
+  }
+}
+
+::google::protobuf::Metadata Jar::GetMetadata() const {
+  protobuf_AssignDescriptorsOnce();
+  ::google::protobuf::Metadata metadata;
+  metadata.descriptor = Jar_descriptor_;
+  metadata.reflection = Jar_reflection_;
+  return metadata;
+}
+
+
 // @@protoc_insertion_point(namespace_scope)
 
 }  // namespace shared

http://git-wip-us.apache.org/repos/asf/drill/blob/2558803e/contrib/native/client/src/protobuf/UserBitShared.pb.h
----------------------------------------------------------------------
diff --git a/contrib/native/client/src/protobuf/UserBitShared.pb.h b/contrib/native/client/src/protobuf/UserBitShared.pb.h
index 41279ca..82db444 100644
--- a/contrib/native/client/src/protobuf/UserBitShared.pb.h
+++ b/contrib/native/client/src/protobuf/UserBitShared.pb.h
@@ -57,6 +57,8 @@ class MinorFragmentProfile;
 class OperatorProfile;
 class StreamProfile;
 class MetricValue;
+class Registry;
+class Jar;
 
 enum DrillPBError_ErrorType {
   DrillPBError_ErrorType_CONNECTION = 0,
@@ -152,11 +154,13 @@ inline bool RpcChannel_Parse(
 enum QueryType {
   SQL = 1,
   LOGICAL = 2,
-  PHYSICAL = 3
+  PHYSICAL = 3,
+  EXECUTION = 4,
+  PREPARED_STATEMENT = 5
 };
 bool QueryType_IsValid(int value);
 const QueryType QueryType_MIN = SQL;
-const QueryType QueryType_MAX = PHYSICAL;
+const QueryType QueryType_MAX = PREPARED_STATEMENT;
 const int QueryType_ARRAYSIZE = QueryType_MAX + 1;
 
 const ::google::protobuf::EnumDescriptor* QueryType_descriptor();
@@ -2751,6 +2755,195 @@ class MetricValue : public ::google::protobuf::Message {
   void InitAsDefaultInstance();
   static MetricValue* default_instance_;
 };
+// -------------------------------------------------------------------
+
+class Registry : public ::google::protobuf::Message {
+ public:
+  Registry();
+  virtual ~Registry();
+
+  Registry(const Registry& from);
+
+  inline Registry& operator=(const Registry& from) {
+    CopyFrom(from);
+    return *this;
+  }
+
+  inline const ::google::protobuf::UnknownFieldSet& unknown_fields() const {
+    return _unknown_fields_;
+  }
+
+  inline ::google::protobuf::UnknownFieldSet* mutable_unknown_fields() {
+    return &_unknown_fields_;
+  }
+
+  static const ::google::protobuf::Descriptor* descriptor();
+  static const Registry& default_instance();
+
+  void Swap(Registry* other);
+
+  // implements Message ----------------------------------------------
+
+  Registry* New() const;
+  void CopyFrom(const ::google::protobuf::Message& from);
+  void MergeFrom(const ::google::protobuf::Message& from);
+  void CopyFrom(const Registry& from);
+  void MergeFrom(const Registry& from);
+  void Clear();
+  bool IsInitialized() const;
+
+  int ByteSize() const;
+  bool MergePartialFromCodedStream(
+      ::google::protobuf::io::CodedInputStream* input);
+  void SerializeWithCachedSizes(
+      ::google::protobuf::io::CodedOutputStream* output) const;
+  ::google::protobuf::uint8* SerializeWithCachedSizesToArray(::google::protobuf::uint8* output) const;
+  int GetCachedSize() const { return _cached_size_; }
+  private:
+  void SharedCtor();
+  void SharedDtor();
+  void SetCachedSize(int size) const;
+  public:
+
+  ::google::protobuf::Metadata GetMetadata() const;
+
+  // nested types ----------------------------------------------------
+
+  // accessors -------------------------------------------------------
+
+  // repeated .exec.shared.Jar jar = 1;
+  inline int jar_size() const;
+  inline void clear_jar();
+  static const int kJarFieldNumber = 1;
+  inline const ::exec::shared::Jar& jar(int index) const;
+  inline ::exec::shared::Jar* mutable_jar(int index);
+  inline ::exec::shared::Jar* add_jar();
+  inline const ::google::protobuf::RepeatedPtrField< ::exec::shared::Jar >&
+      jar() const;
+  inline ::google::protobuf::RepeatedPtrField< ::exec::shared::Jar >*
+      mutable_jar();
+
+  // @@protoc_insertion_point(class_scope:exec.shared.Registry)
+ private:
+
+  ::google::protobuf::UnknownFieldSet _unknown_fields_;
+
+  ::google::protobuf::RepeatedPtrField< ::exec::shared::Jar > jar_;
+
+  mutable int _cached_size_;
+  ::google::protobuf::uint32 _has_bits_[(1 + 31) / 32];
+
+  friend void  protobuf_AddDesc_UserBitShared_2eproto();
+  friend void protobuf_AssignDesc_UserBitShared_2eproto();
+  friend void protobuf_ShutdownFile_UserBitShared_2eproto();
+
+  void InitAsDefaultInstance();
+  static Registry* default_instance_;
+};
+// -------------------------------------------------------------------
+
+class Jar : public ::google::protobuf::Message {
+ public:
+  Jar();
+  virtual ~Jar();
+
+  Jar(const Jar& from);
+
+  inline Jar& operator=(const Jar& from) {
+    CopyFrom(from);
+    return *this;
+  }
+
+  inline const ::google::protobuf::UnknownFieldSet& unknown_fields() const {
+    return _unknown_fields_;
+  }
+
+  inline ::google::protobuf::UnknownFieldSet* mutable_unknown_fields() {
+    return &_unknown_fields_;
+  }
+
+  static const ::google::protobuf::Descriptor* descriptor();
+  static const Jar& default_instance();
+
+  void Swap(Jar* other);
+
+  // implements Message ----------------------------------------------
+
+  Jar* New() const;
+  void CopyFrom(const ::google::protobuf::Message& from);
+  void MergeFrom(const ::google::protobuf::Message& from);
+  void CopyFrom(const Jar& from);
+  void MergeFrom(const Jar& from);
+  void Clear();
+  bool IsInitialized() const;
+
+  int ByteSize() const;
+  bool MergePartialFromCodedStream(
+      ::google::protobuf::io::CodedInputStream* input);
+  void SerializeWithCachedSizes(
+      ::google::protobuf::io::CodedOutputStream* output) const;
+  ::google::protobuf::uint8* SerializeWithCachedSizesToArray(::google::protobuf::uint8* output) const;
+  int GetCachedSize() const { return _cached_size_; }
+  private:
+  void SharedCtor();
+  void SharedDtor();
+  void SetCachedSize(int size) const;
+  public:
+
+  ::google::protobuf::Metadata GetMetadata() const;
+
+  // nested types ----------------------------------------------------
+
+  // accessors -------------------------------------------------------
+
+  // optional string name = 1;
+  inline bool has_name() const;
+  inline void clear_name();
+  static const int kNameFieldNumber = 1;
+  inline const ::std::string& name() const;
+  inline void set_name(const ::std::string& value);
+  inline void set_name(const char* value);
+  inline void set_name(const char* value, size_t size);
+  inline ::std::string* mutable_name();
+  inline ::std::string* release_name();
+  inline void set_allocated_name(::std::string* name);
+
+  // repeated string function_signature = 2;
+  inline int function_signature_size() const;
+  inline void clear_function_signature();
+  static const int kFunctionSignatureFieldNumber = 2;
+  inline const ::std::string& function_signature(int index) const;
+  inline ::std::string* mutable_function_signature(int index);
+  inline void set_function_signature(int index, const ::std::string& value);
+  inline void set_function_signature(int index, const char* value);
+  inline void set_function_signature(int index, const char* value, size_t size);
+  inline ::std::string* add_function_signature();
+  inline void add_function_signature(const ::std::string& value);
+  inline void add_function_signature(const char* value);
+  inline void add_function_signature(const char* value, size_t size);
+  inline const ::google::protobuf::RepeatedPtrField< ::std::string>& function_signature() const;
+  inline ::google::protobuf::RepeatedPtrField< ::std::string>* mutable_function_signature();
+
+  // @@protoc_insertion_point(class_scope:exec.shared.Jar)
+ private:
+  inline void set_has_name();
+  inline void clear_has_name();
+
+  ::google::protobuf::UnknownFieldSet _unknown_fields_;
+
+  ::std::string* name_;
+  ::google::protobuf::RepeatedPtrField< ::std::string> function_signature_;
+
+  mutable int _cached_size_;
+  ::google::protobuf::uint32 _has_bits_[(2 + 31) / 32];
+
+  friend void  protobuf_AddDesc_UserBitShared_2eproto();
+  friend void protobuf_AssignDesc_UserBitShared_2eproto();
+  friend void protobuf_ShutdownFile_UserBitShared_2eproto();
+
+  void InitAsDefaultInstance();
+  static Jar* default_instance_;
+};
 // ===================================================================
 
 
@@ -5935,6 +6128,153 @@ inline void MetricValue::set_double_value(double value) {
   double_value_ = value;
 }
 
+// -------------------------------------------------------------------
+
+// Registry
+
+// repeated .exec.shared.Jar jar = 1;
+inline int Registry::jar_size() const {
+  return jar_.size();
+}
+inline void Registry::clear_jar() {
+  jar_.Clear();
+}
+inline const ::exec::shared::Jar& Registry::jar(int index) const {
+  return jar_.Get(index);
+}
+inline ::exec::shared::Jar* Registry::mutable_jar(int index) {
+  return jar_.Mutable(index);
+}
+inline ::exec::shared::Jar* Registry::add_jar() {
+  return jar_.Add();
+}
+inline const ::google::protobuf::RepeatedPtrField< ::exec::shared::Jar >&
+Registry::jar() const {
+  return jar_;
+}
+inline ::google::protobuf::RepeatedPtrField< ::exec::shared::Jar >*
+Registry::mutable_jar() {
+  return &jar_;
+}
+
+// -------------------------------------------------------------------
+
+// Jar
+
+// optional string name = 1;
+inline bool Jar::has_name() const {
+  return (_has_bits_[0] & 0x00000001u) != 0;
+}
+inline void Jar::set_has_name() {
+  _has_bits_[0] |= 0x00000001u;
+}
+inline void Jar::clear_has_name() {
+  _has_bits_[0] &= ~0x00000001u;
+}
+inline void Jar::clear_name() {
+  if (name_ != &::google::protobuf::internal::kEmptyString) {
+    name_->clear();
+  }
+  clear_has_name();
+}
+inline const ::std::string& Jar::name() const {
+  return *name_;
+}
+inline void Jar::set_name(const ::std::string& value) {
+  set_has_name();
+  if (name_ == &::google::protobuf::internal::kEmptyString) {
+    name_ = new ::std::string;
+  }
+  name_->assign(value);
+}
+inline void Jar::set_name(const char* value) {
+  set_has_name();
+  if (name_ == &::google::protobuf::internal::kEmptyString) {
+    name_ = new ::std::string;
+  }
+  name_->assign(value);
+}
+inline void Jar::set_name(const char* value, size_t size) {
+  set_has_name();
+  if (name_ == &::google::protobuf::internal::kEmptyString) {
+    name_ = new ::std::string;
+  }
+  name_->assign(reinterpret_cast<const char*>(value), size);
+}
+inline ::std::string* Jar::mutable_name() {
+  set_has_name();
+  if (name_ == &::google::protobuf::internal::kEmptyString) {
+    name_ = new ::std::string;
+  }
+  return name_;
+}
+inline ::std::string* Jar::release_name() {
+  clear_has_name();
+  if (name_ == &::google::protobuf::internal::kEmptyString) {
+    return NULL;
+  } else {
+    ::std::string* temp = name_;
+    name_ = const_cast< ::std::string*>(&::google::protobuf::internal::kEmptyString);
+    return temp;
+  }
+}
+inline void Jar::set_allocated_name(::std::string* name) {
+  if (name_ != &::google::protobuf::internal::kEmptyString) {
+    delete name_;
+  }
+  if (name) {
+    set_has_name();
+    name_ = name;
+  } else {
+    clear_has_name();
+    name_ = const_cast< ::std::string*>(&::google::protobuf::internal::kEmptyString);
+  }
+}
+
+// repeated string function_signature = 2;
+inline int Jar::function_signature_size() const {
+  return function_signature_.size();
+}
+inline void Jar::clear_function_signature() {
+  function_signature_.Clear();
+}
+inline const ::std::string& Jar::function_signature(int index) const {
+  return function_signature_.Get(index);
+}
+inline ::std::string* Jar::mutable_function_signature(int index) {
+  return function_signature_.Mutable(index);
+}
+inline void Jar::set_function_signature(int index, const ::std::string& value) {
+  function_signature_.Mutable(index)->assign(value);
+}
+inline void Jar::set_function_signature(int index, const char* value) {
+  function_signature_.Mutable(index)->assign(value);
+}
+inline void Jar::set_function_signature(int index, const char* value, size_t size) {
+  function_signature_.Mutable(index)->assign(
+    reinterpret_cast<const char*>(value), size);
+}
+inline ::std::string* Jar::add_function_signature() {
+  return function_signature_.Add();
+}
+inline void Jar::add_function_signature(const ::std::string& value) {
+  function_signature_.Add()->assign(value);
+}
+inline void Jar::add_function_signature(const char* value) {
+  function_signature_.Add()->assign(value);
+}
+inline void Jar::add_function_signature(const char* value, size_t size) {
+  function_signature_.Add()->assign(reinterpret_cast<const char*>(value), size);
+}
+inline const ::google::protobuf::RepeatedPtrField< ::std::string>&
+Jar::function_signature() const {
+  return function_signature_;
+}
+inline ::google::protobuf::RepeatedPtrField< ::std::string>*
+Jar::mutable_function_signature() {
+  return &function_signature_;
+}
+
 
 // @@protoc_insertion_point(namespace_scope)