You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by cw...@apache.org on 2014/01/03 02:34:10 UTC

svn commit: r1554989 [1/6] - in /hive/trunk: itests/hive-unit/src/test/java/org/apache/hive/jdbc/miniHS2/ jdbc/src/java/org/apache/hive/jdbc/ ql/src/java/org/apache/hadoop/hive/ql/exec/ serde/src/java/org/apache/hadoop/hive/serde2/ service/if/ service/...

Author: cws
Date: Fri Jan  3 01:34:08 2014
New Revision: 1554989

URL: http://svn.apache.org/r1554989
Log:
HIVE-3746: Fix HS2 ResultSet Serialization Performance Regression (Navis via cws)

Added:
    hive/trunk/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TBinaryColumn.java
    hive/trunk/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TBoolColumn.java
    hive/trunk/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TByteColumn.java
    hive/trunk/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TDoubleColumn.java
    hive/trunk/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TI16Column.java
    hive/trunk/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TI32Column.java
    hive/trunk/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TI64Column.java
    hive/trunk/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TStringColumn.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/Column.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/ColumnBasedSet.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/RowBasedSet.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/RowSetFactory.java
Modified:
    hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/jdbc/miniHS2/TestHiveServer2.java
    hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HiveBaseResultSet.java
    hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java
    hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HiveDatabaseMetaData.java
    hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HiveQueryResultSet.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchFormatter.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/ListSinkOperator.java
    hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/SerDeUtils.java
    hive/trunk/service/if/TCLIService.thrift
    hive/trunk/service/src/gen/thrift/gen-cpp/TCLIService_types.cpp
    hive/trunk/service/src/gen/thrift/gen-cpp/TCLIService_types.h
    hive/trunk/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TColumn.java
    hive/trunk/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TExecuteStatementReq.java
    hive/trunk/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetTablesReq.java
    hive/trunk/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TOpenSessionReq.java
    hive/trunk/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TOpenSessionResp.java
    hive/trunk/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TProtocolVersion.java
    hive/trunk/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TRowSet.java
    hive/trunk/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TStatus.java
    hive/trunk/service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TTypeQualifiers.java
    hive/trunk/service/src/gen/thrift/gen-py/TCLIService/ttypes.py
    hive/trunk/service/src/gen/thrift/gen-rb/t_c_l_i_service_types.rb
    hive/trunk/service/src/java/org/apache/hive/service/cli/CLIService.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/ColumnValue.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/OperationHandle.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/RowSet.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/SessionHandle.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/TableSchema.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/operation/GetTablesOperation.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/operation/Operation.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSession.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/session/SessionManager.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/thrift/EmbeddedThriftBinaryCLIService.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java
    hive/trunk/service/src/test/org/apache/hive/service/cli/thrift/ThriftCLIServiceTest.java

Modified: hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/jdbc/miniHS2/TestHiveServer2.java
URL: http://svn.apache.org/viewvc/hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/jdbc/miniHS2/TestHiveServer2.java?rev=1554989&r1=1554988&r2=1554989&view=diff
==============================================================================
--- hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/jdbc/miniHS2/TestHiveServer2.java (original)
+++ hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/jdbc/miniHS2/TestHiveServer2.java Fri Jan  3 01:34:08 2014
@@ -64,6 +64,6 @@ public class TestHiveServer2 {
     serviceClient.executeStatement(sessHandle, "CREATE TABLE " + tabName + " (id INT)", confOverlay);
     OperationHandle opHandle = serviceClient.executeStatement(sessHandle, "SHOW TABLES", confOverlay);
     RowSet rowSet = serviceClient.fetchResults(opHandle);
-    assertFalse(rowSet.getSize() == 0);
+    assertFalse(rowSet.numRows() == 0);
   }
 }

Modified: hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HiveBaseResultSet.java
URL: http://svn.apache.org/viewvc/hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HiveBaseResultSet.java?rev=1554989&r1=1554988&r2=1554989&view=diff
==============================================================================
--- hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HiveBaseResultSet.java (original)
+++ hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HiveBaseResultSet.java Fri Jan  3 01:34:08 2014
@@ -44,25 +44,17 @@ import java.util.Map;
 
 import org.apache.hive.service.cli.TableSchema;
 import org.apache.hive.service.cli.Type;
-import org.apache.hive.service.cli.thrift.TBoolValue;
-import org.apache.hive.service.cli.thrift.TByteValue;
-import org.apache.hive.service.cli.thrift.TColumnValue;
-import org.apache.hive.service.cli.thrift.TDoubleValue;
-import org.apache.hive.service.cli.thrift.TI16Value;
-import org.apache.hive.service.cli.thrift.TI32Value;
-import org.apache.hive.service.cli.thrift.TI64Value;
-import org.apache.hive.service.cli.thrift.TRow;
-import org.apache.hive.service.cli.thrift.TStringValue;
 
 /**
  * Data independent base class which implements the common part of
  * all Hive result sets.
  */
 public abstract class HiveBaseResultSet implements ResultSet {
+
   protected Statement statement = null;
   protected SQLWarning warningChain = null;
   protected boolean wasNull = false;
-  protected TRow row;
+  protected Object[] row;
   protected List<String> columnNames;
   protected List<String> columnTypes;
   protected List<JdbcColumnAttributes> columnAttributes;
@@ -380,176 +372,47 @@ public abstract class HiveBaseResultSet 
     throw new SQLException("Method not supported");
   }
 
-  private Boolean getBooleanValue(TBoolValue tBoolValue) {
-    if (tBoolValue.isSetValue()) {
-      wasNull = false;
-      return tBoolValue.isValue();
-    }
-    wasNull = true;
-    return null;
-  }
-
-  private Byte getByteValue(TByteValue tByteValue) {
-    if (tByteValue.isSetValue()) {
-      wasNull = false;
-      return tByteValue.getValue();
-    }
-    wasNull = true;
-    return null;
-  }
-
-  private Short getShortValue(TI16Value tI16Value) {
-    if (tI16Value.isSetValue()) {
-      wasNull = false;
-      return tI16Value.getValue();
-    }
-    wasNull = true;
-    return null;
-  }
-
-  private Integer getIntegerValue(TI32Value tI32Value) {
-    if (tI32Value.isSetValue()) {
-      wasNull = false;
-      return tI32Value.getValue();
-    }
-    wasNull = true;
-    return null;
-  }
-
-  private Long getLongValue(TI64Value tI64Value) {
-    if (tI64Value.isSetValue()) {
-      wasNull = false;
-      return tI64Value.getValue();
-    }
-    wasNull = true;
-    return null;
-  }
-
-  private Double getDoubleValue(TDoubleValue tDoubleValue) {
-    if (tDoubleValue.isSetValue()) {
-      wasNull = false;
-      return tDoubleValue.getValue();
-    }
-    wasNull = true;
-    return null;
-  }
-
-  private String getStringValue(TStringValue tStringValue) {
-    if (tStringValue.isSetValue()) {
-      wasNull = false;
-      return tStringValue.getValue();
-    }
-    wasNull = true;
-    return null;
-  }
-
-  private Date getDateValue(TStringValue tStringValue) {
-    if (tStringValue.isSetValue()) {
-      wasNull = false;
-      return Date.valueOf(tStringValue.getValue());
-    }
-    wasNull = true;
-    return null;
-  }
-
-  private Timestamp getTimestampValue(TStringValue tStringValue) {
-    if (tStringValue.isSetValue()) {
-      wasNull = false;
-      return Timestamp.valueOf(tStringValue.getValue());
-    }
-    wasNull = true;
-    return null;
-  }
-
-  private byte[] getBinaryValue(TStringValue tString) {
-    if (tString.isSetValue()) {
-      wasNull = false;
-      return tString.getValue().getBytes();
-    }
-    wasNull = true;
-    return null;
-  }
-
-  private BigDecimal getBigDecimalValue(TStringValue tStringValue) {
-    if (tStringValue.isSetValue()) {
-      wasNull = false;
-      return new BigDecimal(tStringValue.getValue());
-    }
-    wasNull = true;
-    return null;
-  }
-
   private Object getColumnValue(int columnIndex) throws SQLException {
     if (row == null) {
       throw new SQLException("No row found.");
     }
-    List<TColumnValue> colVals = row.getColVals();
-    if (colVals == null) {
+    if (row.length == 0) {
       throw new SQLException("RowSet does not contain any columns!");
     }
-    if (columnIndex > colVals.size()) {
+    if (columnIndex > row.length) {
       throw new SQLException("Invalid columnIndex: " + columnIndex);
     }
-
-    TColumnValue tColumnValue = colVals.get(columnIndex - 1);
     Type columnType = getSchema().getColumnDescriptorAt(columnIndex - 1).getType();
 
-    switch (columnType) {
-    case BOOLEAN_TYPE:
-      return getBooleanValue(tColumnValue.getBoolVal());
-    case TINYINT_TYPE:
-      return getByteValue(tColumnValue.getByteVal());
-    case SMALLINT_TYPE:
-      return getShortValue(tColumnValue.getI16Val());
-    case INT_TYPE:
-      return getIntegerValue(tColumnValue.getI32Val());
-    case BIGINT_TYPE:
-      return getLongValue(tColumnValue.getI64Val());
-    case FLOAT_TYPE:
-      return getDoubleValue(tColumnValue.getDoubleVal());
-    case DOUBLE_TYPE:
-      return getDoubleValue(tColumnValue.getDoubleVal());
-    case STRING_TYPE:
-      return getStringValue(tColumnValue.getStringVal());
-    case CHAR_TYPE:
-      return getStringValue(tColumnValue.getStringVal());
-    case VARCHAR_TYPE:
-      return getStringValue(tColumnValue.getStringVal());
-    case BINARY_TYPE:
-      return getBinaryValue(tColumnValue.getStringVal());
-    case DATE_TYPE:
-      return getDateValue(tColumnValue.getStringVal());
-    case TIMESTAMP_TYPE:
-      return getTimestampValue(tColumnValue.getStringVal());
-    case DECIMAL_TYPE:
-      return getBigDecimalValue(tColumnValue.getStringVal());
-    case NULL_TYPE:
-      wasNull = true;
-      return null;
-    default:
-      throw new SQLException("Unrecognized column type:" + columnType);
+    try {
+      Object evaluated = evaluate(columnType, row[columnIndex - 1]);
+      wasNull = evaluated == null;
+      return evaluated;
+    } catch (Exception e) {
+      e.printStackTrace();
+      throw new SQLException("Unrecognized column type:" + columnType, e);
     }
+  }
 
-    /*
-    switch (tColumnValue.getSetField()) {
-    case BOOL_VAL:
-      return getBooleanValue(tColumnValue.getBoolVal());
-    case BYTE_VAL:
-      return getByteValue(tColumnValue.getByteVal());
-    case I16_VAL:
-      return getShortValue(tColumnValue.getI16Val());
-    case I32_VAL:
-      return getIntegerValue(tColumnValue.getI32Val());
-    case I64_VAL:
-      return getLongValue(tColumnValue.getI64Val());
-    case DOUBLE_VAL:
-      return getDoubleValue(tColumnValue.getDoubleVal());
-    case STRING_VAL:
-      return getStringValue(tColumnValue.getStringVal());
-    default:
-      throw new SQLException("Unrecognized column type:" + tColumnValue.getSetField());
+  private Object evaluate(Type type, Object value) {
+    if (value == null) {
+      return null;
+    }
+    switch (type) {
+      case BINARY_TYPE:
+        if (value instanceof String) {
+          return ((String) value).getBytes();
+        }
+        return value;
+      case TIMESTAMP_TYPE:
+        return Timestamp.valueOf((String) value);
+      case DECIMAL_TYPE:
+        return new BigDecimal((String)value);
+      case DATE_TYPE:
+        return Date.valueOf((String) value);
+      default:
+        return value;
     }
-    */
   }
 
   public Object getObject(int columnIndex) throws SQLException {

Modified: hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java
URL: http://svn.apache.org/viewvc/hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java?rev=1554989&r1=1554988&r2=1554989&view=diff
==============================================================================
--- hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java (original)
+++ hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HiveConnection.java Fri Jan  3 01:34:08 2014
@@ -93,12 +93,13 @@ public class HiveConnection implements j
   private final Map<String, String> hiveVarMap;
   private final boolean isEmbeddedMode;
   private TTransport transport;
-  private TCLIService.Iface client;
+  private TCLIService.Iface client;   // todo should be replaced by CliServiceClient
   private boolean isClosed = true;
   private SQLWarning warningChain = null;
   private TSessionHandle sessHandle = null;
   private final List<TProtocolVersion> supportedProtocols = new LinkedList<TProtocolVersion>();
   private int loginTimeout = 0;
+  private TProtocolVersion protocol;
 
   public HiveConnection(String uri, Properties info) throws SQLException {
     setupLoginTimeout();
@@ -138,6 +139,7 @@ public class HiveConnection implements j
     supportedProtocols.add(TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V3);
     supportedProtocols.add(TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V4);
     supportedProtocols.add(TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V5);
+    supportedProtocols.add(TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V6);
 
     // open client session
     openSession();
@@ -268,8 +270,10 @@ public class HiveConnection implements j
       if (!supportedProtocols.contains(openResp.getServerProtocolVersion())) {
         throw new TException("Unsupported Hive2 protocol");
       }
+      protocol = openResp.getServerProtocolVersion();
       sessHandle = openResp.getSessionHandle();
     } catch (TException e) {
+      e.printStackTrace();
       throw new SQLException("Could not establish connection to "
           + jdbcURI + ": " + e.getMessage(), " 08S01", e);
     }
@@ -932,4 +936,7 @@ public class HiveConnection implements j
     throw new SQLException("Method not supported");
   }
 
+  public TProtocolVersion getProtocol() {
+    return protocol;
+  }
 }

Modified: hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HiveDatabaseMetaData.java
URL: http://svn.apache.org/viewvc/hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HiveDatabaseMetaData.java?rev=1554989&r1=1554988&r2=1554989&view=diff
==============================================================================
--- hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HiveDatabaseMetaData.java (original)
+++ hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HiveDatabaseMetaData.java Fri Jan  3 01:34:08 2014
@@ -135,7 +135,7 @@ public class HiveDatabaseMetaData implem
     }
     Utils.verifySuccess(catalogResp.getStatus());
 
-    return new HiveQueryResultSet.Builder(null)
+    return new HiveQueryResultSet.Builder(connection)
     .setClient(client)
     .setSessionHandle(sessHandle)
     .setStmtHandle(catalogResp.getOperationHandle())
@@ -219,7 +219,7 @@ public class HiveDatabaseMetaData implem
     }
     Utils.verifySuccess(colResp.getStatus());
     // build the resultset from response
-    return new HiveQueryResultSet.Builder(null)
+    return new HiveQueryResultSet.Builder(connection)
     .setClient(client)
     .setSessionHandle(sessHandle)
     .setStmtHandle(colResp.getOperationHandle())
@@ -331,7 +331,7 @@ public class HiveDatabaseMetaData implem
     }
     Utils.verifySuccess(funcResp.getStatus());
 
-    return new HiveQueryResultSet.Builder(null)
+    return new HiveQueryResultSet.Builder(connection)
     .setClient(client)
     .setSessionHandle(sessHandle)
     .setStmtHandle(funcResp.getOperationHandle())
@@ -344,7 +344,7 @@ public class HiveDatabaseMetaData implem
 
   public ResultSet getImportedKeys(String catalog, String schema, String table)
       throws SQLException {
-    return new HiveQueryResultSet.Builder(null)
+    return new HiveQueryResultSet.Builder(connection)
     .setClient(client)
     .setEmptyResultSet(true)
     .setSchema(
@@ -486,7 +486,7 @@ public class HiveDatabaseMetaData implem
       throws SQLException {
     // Hive doesn't support primary keys
     // using local schema with empty resultset
-    return new HiveQueryResultSet.Builder(null).setClient(client).setEmptyResultSet(true).
+    return new HiveQueryResultSet.Builder(connection).setClient(client).setEmptyResultSet(true).
         setSchema(Arrays.asList("TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "COLUMN_NAME", "KEY_SEQ", "PK_NAME" ),
             Arrays.asList("STRING",    "STRING",      "STRING",     "STRING",       "INT",  "STRING"))
             .build();
@@ -497,7 +497,7 @@ public class HiveDatabaseMetaData implem
       throws SQLException {
     // Hive doesn't support primary keys
     // using local schema with empty resultset
-    return new HiveQueryResultSet.Builder(null).setClient(client).setEmptyResultSet(true).
+    return new HiveQueryResultSet.Builder(connection).setClient(client).setEmptyResultSet(true).
                   setSchema(
                     Arrays.asList("PROCEDURE_CAT", "PROCEDURE_SCHEM", "PROCEDURE_NAME", "COLUMN_NAME", "COLUMN_TYPE",
                               "DATA_TYPE", "TYPE_NAME", "PRECISION", "LENGTH", "SCALE", "RADIX", "NULLABLE", "REMARKS",
@@ -518,7 +518,7 @@ public class HiveDatabaseMetaData implem
       String procedureNamePattern) throws SQLException {
     // Hive doesn't support primary keys
     // using local schema with empty resultset
-    return new HiveQueryResultSet.Builder(null).setClient(client).setEmptyResultSet(true).
+    return new HiveQueryResultSet.Builder(connection).setClient(client).setEmptyResultSet(true).
                   setSchema(
                     Arrays.asList("PROCEDURE_CAT", "PROCEDURE_SCHEM", "PROCEDURE_NAME", "RESERVERD", "RESERVERD",
                                   "RESERVERD", "REMARKS", "PROCEDURE_TYPE", "SPECIFIC_NAME"),
@@ -572,7 +572,7 @@ public class HiveDatabaseMetaData implem
     }
     Utils.verifySuccess(schemaResp.getStatus());
 
-    return new HiveQueryResultSet.Builder(null)
+    return new HiveQueryResultSet.Builder(connection)
     .setClient(client)
     .setSessionHandle(sessHandle)
     .setStmtHandle(schemaResp.getOperationHandle())
@@ -616,7 +616,7 @@ public class HiveDatabaseMetaData implem
     }
     Utils.verifySuccess(tableTypeResp.getStatus());
 
-    return new HiveQueryResultSet.Builder(null)
+    return new HiveQueryResultSet.Builder(connection)
     .setClient(client)
     .setSessionHandle(sessHandle)
     .setStmtHandle(tableTypeResp.getOperationHandle())
@@ -649,7 +649,7 @@ public class HiveDatabaseMetaData implem
     }
     Utils.verifySuccess(getTableResp.getStatus());
 
-    return new HiveQueryResultSet.Builder(null)
+    return new HiveQueryResultSet.Builder(connection)
     .setClient(client)
     .setSessionHandle(sessHandle)
     .setStmtHandle(getTableResp.getOperationHandle())
@@ -705,7 +705,7 @@ public class HiveDatabaseMetaData implem
       throw new SQLException(e.getMessage(), "08S01", e);
     }
     Utils.verifySuccess(getTypeInfoResp.getStatus());
-    return new HiveQueryResultSet.Builder(null)
+    return new HiveQueryResultSet.Builder(connection)
     .setClient(client)
     .setSessionHandle(sessHandle)
     .setStmtHandle(getTypeInfoResp.getOperationHandle())

Modified: hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HiveQueryResultSet.java
URL: http://svn.apache.org/viewvc/hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HiveQueryResultSet.java?rev=1554989&r1=1554988&r2=1554989&view=diff
==============================================================================
--- hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HiveQueryResultSet.java (original)
+++ hive/trunk/jdbc/src/java/org/apache/hive/jdbc/HiveQueryResultSet.java Fri Jan  3 01:34:08 2014
@@ -20,6 +20,7 @@ package org.apache.hive.jdbc;
 
 import static org.apache.hive.service.cli.thrift.TCLIServiceConstants.TYPE_NAMES;
 
+import java.sql.Connection;
 import java.sql.ResultSet;
 import java.sql.ResultSetMetaData;
 import java.sql.Statement;
@@ -31,6 +32,8 @@ import java.util.List;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hive.service.cli.RowSet;
+import org.apache.hive.service.cli.RowSetFactory;
 import org.apache.hive.service.cli.TableSchema;
 import org.apache.hive.service.cli.thrift.TCLIService;
 import org.apache.hive.service.cli.thrift.TCLIServiceConstants;
@@ -42,7 +45,8 @@ import org.apache.hive.service.cli.thrif
 import org.apache.hive.service.cli.thrift.TGetResultSetMetadataResp;
 import org.apache.hive.service.cli.thrift.TOperationHandle;
 import org.apache.hive.service.cli.thrift.TPrimitiveTypeEntry;
-import org.apache.hive.service.cli.thrift.TRow;
+import org.apache.hive.service.cli.thrift.TProtocolVersion;
+import org.apache.hive.service.cli.thrift.TRowSet;
 import org.apache.hive.service.cli.thrift.TSessionHandle;
 import org.apache.hive.service.cli.thrift.TTableSchema;
 import org.apache.hive.service.cli.thrift.TTypeQualifierValue;
@@ -63,15 +67,19 @@ public class HiveQueryResultSet extends 
   private int fetchSize;
   private int rowsFetched = 0;
 
-  private List<TRow> fetchedRows;
-  private Iterator<TRow> fetchedRowsItr;
+  private RowSet fetchedRows;
+  private Iterator<Object[]> fetchedRowsItr;
   private boolean isClosed = false;
   private boolean emptyResultSet = false;
   private boolean isScrollable = false;
   private boolean fetchFirst = false;
 
+  private final TProtocolVersion protocol;
+
+
   public static class Builder {
 
+    private final Connection connection;
     private final Statement statement;
     private TCLIService.Iface client = null;
     private TOperationHandle stmtHandle = null;
@@ -91,8 +99,14 @@ public class HiveQueryResultSet extends 
     private boolean emptyResultSet = false;
     private boolean isScrollable = false;
 
-    public Builder(Statement statement) {
+    public Builder(Statement statement) throws SQLException {
       this.statement = statement;
+      this.connection = statement.getConnection();
+    }
+
+    public Builder(Connection connection) {
+      this.statement = null;
+      this.connection = connection;
     }
 
     public Builder setClient(TCLIService.Iface client) {
@@ -155,6 +169,10 @@ public class HiveQueryResultSet extends 
     public HiveQueryResultSet build() throws SQLException {
       return new HiveQueryResultSet(this);
     }
+
+    public TProtocolVersion getProtocolVersion() throws SQLException {
+      return ((HiveConnection)connection).getProtocol();
+    }
   }
 
   protected HiveQueryResultSet(Builder builder) throws SQLException {
@@ -178,6 +196,7 @@ public class HiveQueryResultSet extends 
       this.maxRows = builder.maxRows;
     }
     this.isScrollable = builder.isScrollable;
+    this.protocol = builder.getProtocolVersion();
   }
 
   /**
@@ -217,6 +236,7 @@ public class HiveQueryResultSet extends 
    * Retrieve schema from the server
    */
   private void retrieveSchema() throws SQLException {
+    System.err.println("[HiveQueryResultSet/next] 0");
     try {
       TGetResultSetMetadataReq metadataReq = new TGetResultSetMetadataReq(stmtHandle);
       // TODO need session handle
@@ -304,13 +324,14 @@ public class HiveQueryResultSet extends 
         fetchedRowsItr = null;
         fetchFirst = false;
       }
-
       if (fetchedRows == null || !fetchedRowsItr.hasNext()) {
         TFetchResultsReq fetchReq = new TFetchResultsReq(stmtHandle,
             orientation, fetchSize);
         TFetchResultsResp fetchResp = client.FetchResults(fetchReq);
         Utils.verifySuccessWithInfo(fetchResp.getStatus());
-        fetchedRows = fetchResp.getResults().getRows();
+
+        TRowSet results = fetchResp.getResults();
+        fetchedRows = RowSetFactory.create(results, protocol);
         fetchedRowsItr = fetchedRows.iterator();
       }
 

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchFormatter.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchFormatter.java?rev=1554989&r1=1554988&r2=1554989&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchFormatter.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchFormatter.java Fri Jan  3 01:34:08 2014
@@ -42,8 +42,11 @@ public interface FetchFormatter<T> exten
 
   public static class ThriftFormatter implements FetchFormatter<Object> {
 
+    int protocol;
+
     @Override
     public void initialize(Configuration hconf, Properties props) throws Exception {
+      protocol = hconf.getInt(ListSinkOperator.OUTPUT_PROTOCOL, 0);
     }
 
     @Override
@@ -56,7 +59,7 @@ public interface FetchFormatter<T> exten
         StructField fieldRef = fields.get(i);
         Object field = structOI.getStructFieldData(row, fieldRef);
         converted[i] = field == null ? null :
-            SerDeUtils.toThriftPayload(field, fieldRef.getFieldObjectInspector());
+            SerDeUtils.toThriftPayload(field, fieldRef.getFieldObjectInspector(), protocol);
       }
       return converted;
     }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/ListSinkOperator.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/ListSinkOperator.java?rev=1554989&r1=1554988&r2=1554989&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/ListSinkOperator.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/ListSinkOperator.java Fri Jan  3 01:34:08 2014
@@ -36,6 +36,7 @@ import org.apache.hadoop.util.Reflection
 public class ListSinkOperator extends Operator<ListSinkDesc> {
 
   public static final String OUTPUT_FORMATTER = "output.formatter";
+  public static final String OUTPUT_PROTOCOL = "output.protocol";
 
   private transient List res;
   private transient FetchFormatter fetcher;

Modified: hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/SerDeUtils.java
URL: http://svn.apache.org/viewvc/hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/SerDeUtils.java?rev=1554989&r1=1554988&r2=1554989&view=diff
==============================================================================
--- hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/SerDeUtils.java (original)
+++ hive/trunk/serde/src/java/org/apache/hadoop/hive/serde2/SerDeUtils.java Fri Jan  3 01:34:08 2014
@@ -198,11 +198,15 @@ public final class SerDeUtils {
    *
    * This method is kept consistent with {@link HiveResultSetMetaData#hiveTypeToSqlType}.
    */
-  public static Object toThriftPayload(Object val, ObjectInspector valOI) {
+  public static Object toThriftPayload(Object val, ObjectInspector valOI, int version) {
     if (valOI.getCategory() == ObjectInspector.Category.PRIMITIVE) {
+      if (val == null) {
+        return null;
+      }
       Object obj = ObjectInspectorUtils.copyToStandardObject(val, valOI,
           ObjectInspectorUtils.ObjectInspectorCopyOption.JAVA);
-      if (((PrimitiveObjectInspector)valOI).getPrimitiveCategory() ==
+      // uses string type for binary before HIVE_CLI_SERVICE_PROTOCOL_V6
+      if (version < 5 && ((PrimitiveObjectInspector)valOI).getPrimitiveCategory() ==
           PrimitiveObjectInspector.PrimitiveCategory.BINARY) {
         // todo HIVE-5269
         return new String((byte[])obj);

Modified: hive/trunk/service/if/TCLIService.thrift
URL: http://svn.apache.org/viewvc/hive/trunk/service/if/TCLIService.thrift?rev=1554989&r1=1554988&r2=1554989&view=diff
==============================================================================
--- hive/trunk/service/if/TCLIService.thrift (original)
+++ hive/trunk/service/if/TCLIService.thrift Fri Jan  3 01:34:08 2014
@@ -51,6 +51,9 @@ enum TProtocolVersion {
 
   // V5 adds error details when GetOperationStatus returns in error state
   HIVE_CLI_SERVICE_PROTOCOL_V5
+
+  // V6 uses binary type for binary payload (was string) and uses columnar result set
+  HIVE_CLI_SERVICE_PROTOCOL_V6
 }
 
 enum TTypeId {
@@ -88,9 +91,9 @@ const set<TTypeId> PRIMITIVE_TYPES = [
   TTypeId.TIMESTAMP_TYPE,
   TTypeId.BINARY_TYPE,
   TTypeId.DECIMAL_TYPE,
-  TTypeId.NULL_TYPE
-  TTypeId.DATE_TYPE
-  TTypeId.VARCHAR_TYPE
+  TTypeId.NULL_TYPE,
+  TTypeId.DATE_TYPE,
+  TTypeId.VARCHAR_TYPE,
   TTypeId.CHAR_TYPE
 ]
 
@@ -335,6 +338,62 @@ struct TRow {
   1: required list<TColumnValue> colVals
 }
 
+struct TBoolColumn {
+  1: required list<bool> values
+  2: required binary nulls
+}
+
+struct TByteColumn {
+  1: required list<byte> values
+  2: required binary nulls
+}
+
+struct TI16Column {
+  1: required list<i16> values
+  2: required binary nulls
+}
+
+struct TI32Column {
+  1: required list<i32> values
+  2: required binary nulls
+}
+
+struct TI64Column {
+  1: required list<i64> values
+  2: required binary nulls
+}
+
+struct TDoubleColumn {
+  1: required list<double> values
+  2: required binary nulls
+}
+
+struct TStringColumn {
+  1: required list<string> values
+  2: required binary nulls
+}
+
+struct TBinaryColumn {
+  1: required list<binary> values
+  2: required binary nulls
+}
+
+// Note that Hive's type system is richer than Thrift's,
+// so in some cases we have to map multiple Hive types
+// to the same Thrift type. On the client-side this is
+// disambiguated by looking at the Schema of the
+// result set.
+union TColumn {
+  1: TBoolColumn   boolVal      // BOOLEAN
+  2: TByteColumn   byteVal      // TINYINT
+  3: TI16Column    i16Val       // SMALLINT
+  4: TI32Column    i32Val       // INT
+  5: TI64Column    i64Val       // BIGINT, TIMESTAMP
+  6: TDoubleColumn doubleVal    // FLOAT, DOUBLE
+  7: TStringColumn stringVal    // STRING, LIST, MAP, STRUCT, UNIONTYPE, DECIMAL, NULL
+  8: TBinaryColumn binaryVal    // BINARY
+}
+
 // Represents a rowset
 struct TRowSet {
   // The starting row offset of this rowset.
@@ -490,7 +549,7 @@ struct TOperationHandle {
 // which operations may be executed.
 struct TOpenSessionReq {
   // The version of the HiveServer2 protocol that the client is using.
-  1: required TProtocolVersion client_protocol = TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V5
+  1: required TProtocolVersion client_protocol = TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V6
 
   // Username and password for authentication.
   // Depending on the authentication scheme being used,
@@ -509,7 +568,7 @@ struct TOpenSessionResp {
   1: required TStatus status
 
   // The protocol version that the server is using.
-  2: required TProtocolVersion serverProtocolVersion = TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V5
+  2: required TProtocolVersion serverProtocolVersion = TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V6
 
   // Session Handle
   3: optional TSessionHandle sessionHandle

Modified: hive/trunk/service/src/gen/thrift/gen-cpp/TCLIService_types.cpp
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/gen/thrift/gen-cpp/TCLIService_types.cpp?rev=1554989&r1=1554988&r2=1554989&view=diff
==============================================================================
--- hive/trunk/service/src/gen/thrift/gen-cpp/TCLIService_types.cpp (original)
+++ hive/trunk/service/src/gen/thrift/gen-cpp/TCLIService_types.cpp Fri Jan  3 01:34:08 2014
@@ -15,16 +15,18 @@ int _kTProtocolVersionValues[] = {
   TProtocolVersion::HIVE_CLI_SERVICE_PROTOCOL_V2,
   TProtocolVersion::HIVE_CLI_SERVICE_PROTOCOL_V3,
   TProtocolVersion::HIVE_CLI_SERVICE_PROTOCOL_V4,
-  TProtocolVersion::HIVE_CLI_SERVICE_PROTOCOL_V5
+  TProtocolVersion::HIVE_CLI_SERVICE_PROTOCOL_V5,
+  TProtocolVersion::HIVE_CLI_SERVICE_PROTOCOL_V6
 };
 const char* _kTProtocolVersionNames[] = {
   "HIVE_CLI_SERVICE_PROTOCOL_V1",
   "HIVE_CLI_SERVICE_PROTOCOL_V2",
   "HIVE_CLI_SERVICE_PROTOCOL_V3",
   "HIVE_CLI_SERVICE_PROTOCOL_V4",
-  "HIVE_CLI_SERVICE_PROTOCOL_V5"
+  "HIVE_CLI_SERVICE_PROTOCOL_V5",
+  "HIVE_CLI_SERVICE_PROTOCOL_V6"
 };
-const std::map<int, const char*> _TProtocolVersion_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(5, _kTProtocolVersionValues, _kTProtocolVersionNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL));
+const std::map<int, const char*> _TProtocolVersion_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(6, _kTProtocolVersionValues, _kTProtocolVersionNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL));
 
 int _kTTypeIdValues[] = {
   TTypeId::BOOLEAN_TYPE,
@@ -2233,8 +2235,961 @@ void swap(TRow &a, TRow &b) {
   swap(a.colVals, b.colVals);
 }
 
-const char* TRowSet::ascii_fingerprint = "698727A24268879440EE0DAFE68FC1C5";
-const uint8_t TRowSet::binary_fingerprint[16] = {0x69,0x87,0x27,0xA2,0x42,0x68,0x87,0x94,0x40,0xEE,0x0D,0xAF,0xE6,0x8F,0xC1,0xC5};
+const char* TBoolColumn::ascii_fingerprint = "F9058324D96DB7F974D8ACDC01C54219";
+const uint8_t TBoolColumn::binary_fingerprint[16] = {0xF9,0x05,0x83,0x24,0xD9,0x6D,0xB7,0xF9,0x74,0xD8,0xAC,0xDC,0x01,0xC5,0x42,0x19};
+
+uint32_t TBoolColumn::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+  bool isset_values = false;
+  bool isset_nulls = false;
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            this->values.clear();
+            uint32_t _size85;
+            ::apache::thrift::protocol::TType _etype88;
+            xfer += iprot->readListBegin(_etype88, _size85);
+            this->values.resize(_size85);
+            uint32_t _i89;
+            for (_i89 = 0; _i89 < _size85; ++_i89)
+            {
+              xfer += iprot->readBool(this->values[_i89]);
+            }
+            xfer += iprot->readListEnd();
+          }
+          isset_values = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readBinary(this->nulls);
+          isset_nulls = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  if (!isset_values)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  if (!isset_nulls)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  return xfer;
+}
+
+uint32_t TBoolColumn::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  xfer += oprot->writeStructBegin("TBoolColumn");
+
+  xfer += oprot->writeFieldBegin("values", ::apache::thrift::protocol::T_LIST, 1);
+  {
+    xfer += oprot->writeListBegin(::apache::thrift::protocol::T_BOOL, static_cast<uint32_t>(this->values.size()));
+    std::vector<bool> ::const_iterator _iter90;
+    for (_iter90 = this->values.begin(); _iter90 != this->values.end(); ++_iter90)
+    {
+      xfer += oprot->writeBool((*_iter90));
+    }
+    xfer += oprot->writeListEnd();
+  }
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("nulls", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeBinary(this->nulls);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+void swap(TBoolColumn &a, TBoolColumn &b) {
+  using ::std::swap;
+  swap(a.values, b.values);
+  swap(a.nulls, b.nulls);
+}
+
+const char* TByteColumn::ascii_fingerprint = "1CB300106BAA463A70BB2A2395900F48";
+const uint8_t TByteColumn::binary_fingerprint[16] = {0x1C,0xB3,0x00,0x10,0x6B,0xAA,0x46,0x3A,0x70,0xBB,0x2A,0x23,0x95,0x90,0x0F,0x48};
+
+uint32_t TByteColumn::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+  bool isset_values = false;
+  bool isset_nulls = false;
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            this->values.clear();
+            uint32_t _size91;
+            ::apache::thrift::protocol::TType _etype94;
+            xfer += iprot->readListBegin(_etype94, _size91);
+            this->values.resize(_size91);
+            uint32_t _i95;
+            for (_i95 = 0; _i95 < _size91; ++_i95)
+            {
+              xfer += iprot->readByte(this->values[_i95]);
+            }
+            xfer += iprot->readListEnd();
+          }
+          isset_values = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readBinary(this->nulls);
+          isset_nulls = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  if (!isset_values)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  if (!isset_nulls)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  return xfer;
+}
+
+uint32_t TByteColumn::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  xfer += oprot->writeStructBegin("TByteColumn");
+
+  xfer += oprot->writeFieldBegin("values", ::apache::thrift::protocol::T_LIST, 1);
+  {
+    xfer += oprot->writeListBegin(::apache::thrift::protocol::T_BYTE, static_cast<uint32_t>(this->values.size()));
+    std::vector<int8_t> ::const_iterator _iter96;
+    for (_iter96 = this->values.begin(); _iter96 != this->values.end(); ++_iter96)
+    {
+      xfer += oprot->writeByte((*_iter96));
+    }
+    xfer += oprot->writeListEnd();
+  }
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("nulls", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeBinary(this->nulls);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+void swap(TByteColumn &a, TByteColumn &b) {
+  using ::std::swap;
+  swap(a.values, b.values);
+  swap(a.nulls, b.nulls);
+}
+
+const char* TI16Column::ascii_fingerprint = "6574CDB1F121C8DB47FB257A3F104BDB";
+const uint8_t TI16Column::binary_fingerprint[16] = {0x65,0x74,0xCD,0xB1,0xF1,0x21,0xC8,0xDB,0x47,0xFB,0x25,0x7A,0x3F,0x10,0x4B,0xDB};
+
+uint32_t TI16Column::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+  bool isset_values = false;
+  bool isset_nulls = false;
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            this->values.clear();
+            uint32_t _size97;
+            ::apache::thrift::protocol::TType _etype100;
+            xfer += iprot->readListBegin(_etype100, _size97);
+            this->values.resize(_size97);
+            uint32_t _i101;
+            for (_i101 = 0; _i101 < _size97; ++_i101)
+            {
+              xfer += iprot->readI16(this->values[_i101]);
+            }
+            xfer += iprot->readListEnd();
+          }
+          isset_values = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readBinary(this->nulls);
+          isset_nulls = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  if (!isset_values)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  if (!isset_nulls)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  return xfer;
+}
+
+uint32_t TI16Column::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  xfer += oprot->writeStructBegin("TI16Column");
+
+  xfer += oprot->writeFieldBegin("values", ::apache::thrift::protocol::T_LIST, 1);
+  {
+    xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I16, static_cast<uint32_t>(this->values.size()));
+    std::vector<int16_t> ::const_iterator _iter102;
+    for (_iter102 = this->values.begin(); _iter102 != this->values.end(); ++_iter102)
+    {
+      xfer += oprot->writeI16((*_iter102));
+    }
+    xfer += oprot->writeListEnd();
+  }
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("nulls", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeBinary(this->nulls);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+void swap(TI16Column &a, TI16Column &b) {
+  using ::std::swap;
+  swap(a.values, b.values);
+  swap(a.nulls, b.nulls);
+}
+
+const char* TI32Column::ascii_fingerprint = "CCCCE89C7E9DA10280F5663700677313";
+const uint8_t TI32Column::binary_fingerprint[16] = {0xCC,0xCC,0xE8,0x9C,0x7E,0x9D,0xA1,0x02,0x80,0xF5,0x66,0x37,0x00,0x67,0x73,0x13};
+
+uint32_t TI32Column::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+  bool isset_values = false;
+  bool isset_nulls = false;
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            this->values.clear();
+            uint32_t _size103;
+            ::apache::thrift::protocol::TType _etype106;
+            xfer += iprot->readListBegin(_etype106, _size103);
+            this->values.resize(_size103);
+            uint32_t _i107;
+            for (_i107 = 0; _i107 < _size103; ++_i107)
+            {
+              xfer += iprot->readI32(this->values[_i107]);
+            }
+            xfer += iprot->readListEnd();
+          }
+          isset_values = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readBinary(this->nulls);
+          isset_nulls = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  if (!isset_values)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  if (!isset_nulls)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  return xfer;
+}
+
+uint32_t TI32Column::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  xfer += oprot->writeStructBegin("TI32Column");
+
+  xfer += oprot->writeFieldBegin("values", ::apache::thrift::protocol::T_LIST, 1);
+  {
+    xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I32, static_cast<uint32_t>(this->values.size()));
+    std::vector<int32_t> ::const_iterator _iter108;
+    for (_iter108 = this->values.begin(); _iter108 != this->values.end(); ++_iter108)
+    {
+      xfer += oprot->writeI32((*_iter108));
+    }
+    xfer += oprot->writeListEnd();
+  }
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("nulls", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeBinary(this->nulls);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+void swap(TI32Column &a, TI32Column &b) {
+  using ::std::swap;
+  swap(a.values, b.values);
+  swap(a.nulls, b.nulls);
+}
+
+const char* TI64Column::ascii_fingerprint = "925353917FC0AF87976A2338011F5A31";
+const uint8_t TI64Column::binary_fingerprint[16] = {0x92,0x53,0x53,0x91,0x7F,0xC0,0xAF,0x87,0x97,0x6A,0x23,0x38,0x01,0x1F,0x5A,0x31};
+
+uint32_t TI64Column::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+  bool isset_values = false;
+  bool isset_nulls = false;
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            this->values.clear();
+            uint32_t _size109;
+            ::apache::thrift::protocol::TType _etype112;
+            xfer += iprot->readListBegin(_etype112, _size109);
+            this->values.resize(_size109);
+            uint32_t _i113;
+            for (_i113 = 0; _i113 < _size109; ++_i113)
+            {
+              xfer += iprot->readI64(this->values[_i113]);
+            }
+            xfer += iprot->readListEnd();
+          }
+          isset_values = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readBinary(this->nulls);
+          isset_nulls = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  if (!isset_values)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  if (!isset_nulls)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  return xfer;
+}
+
+uint32_t TI64Column::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  xfer += oprot->writeStructBegin("TI64Column");
+
+  xfer += oprot->writeFieldBegin("values", ::apache::thrift::protocol::T_LIST, 1);
+  {
+    xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast<uint32_t>(this->values.size()));
+    std::vector<int64_t> ::const_iterator _iter114;
+    for (_iter114 = this->values.begin(); _iter114 != this->values.end(); ++_iter114)
+    {
+      xfer += oprot->writeI64((*_iter114));
+    }
+    xfer += oprot->writeListEnd();
+  }
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("nulls", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeBinary(this->nulls);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+void swap(TI64Column &a, TI64Column &b) {
+  using ::std::swap;
+  swap(a.values, b.values);
+  swap(a.nulls, b.nulls);
+}
+
+const char* TDoubleColumn::ascii_fingerprint = "8FF1C050A8D7FD247AEB23CD71539C09";
+const uint8_t TDoubleColumn::binary_fingerprint[16] = {0x8F,0xF1,0xC0,0x50,0xA8,0xD7,0xFD,0x24,0x7A,0xEB,0x23,0xCD,0x71,0x53,0x9C,0x09};
+
+uint32_t TDoubleColumn::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+  bool isset_values = false;
+  bool isset_nulls = false;
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            this->values.clear();
+            uint32_t _size115;
+            ::apache::thrift::protocol::TType _etype118;
+            xfer += iprot->readListBegin(_etype118, _size115);
+            this->values.resize(_size115);
+            uint32_t _i119;
+            for (_i119 = 0; _i119 < _size115; ++_i119)
+            {
+              xfer += iprot->readDouble(this->values[_i119]);
+            }
+            xfer += iprot->readListEnd();
+          }
+          isset_values = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readBinary(this->nulls);
+          isset_nulls = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  if (!isset_values)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  if (!isset_nulls)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  return xfer;
+}
+
+uint32_t TDoubleColumn::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  xfer += oprot->writeStructBegin("TDoubleColumn");
+
+  xfer += oprot->writeFieldBegin("values", ::apache::thrift::protocol::T_LIST, 1);
+  {
+    xfer += oprot->writeListBegin(::apache::thrift::protocol::T_DOUBLE, static_cast<uint32_t>(this->values.size()));
+    std::vector<double> ::const_iterator _iter120;
+    for (_iter120 = this->values.begin(); _iter120 != this->values.end(); ++_iter120)
+    {
+      xfer += oprot->writeDouble((*_iter120));
+    }
+    xfer += oprot->writeListEnd();
+  }
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("nulls", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeBinary(this->nulls);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+void swap(TDoubleColumn &a, TDoubleColumn &b) {
+  using ::std::swap;
+  swap(a.values, b.values);
+  swap(a.nulls, b.nulls);
+}
+
+const char* TStringColumn::ascii_fingerprint = "BE556BF7091B2DABBA1863D5E458B15F";
+const uint8_t TStringColumn::binary_fingerprint[16] = {0xBE,0x55,0x6B,0xF7,0x09,0x1B,0x2D,0xAB,0xBA,0x18,0x63,0xD5,0xE4,0x58,0xB1,0x5F};
+
+uint32_t TStringColumn::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+  bool isset_values = false;
+  bool isset_nulls = false;
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            this->values.clear();
+            uint32_t _size121;
+            ::apache::thrift::protocol::TType _etype124;
+            xfer += iprot->readListBegin(_etype124, _size121);
+            this->values.resize(_size121);
+            uint32_t _i125;
+            for (_i125 = 0; _i125 < _size121; ++_i125)
+            {
+              xfer += iprot->readString(this->values[_i125]);
+            }
+            xfer += iprot->readListEnd();
+          }
+          isset_values = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readBinary(this->nulls);
+          isset_nulls = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  if (!isset_values)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  if (!isset_nulls)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  return xfer;
+}
+
+uint32_t TStringColumn::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  xfer += oprot->writeStructBegin("TStringColumn");
+
+  xfer += oprot->writeFieldBegin("values", ::apache::thrift::protocol::T_LIST, 1);
+  {
+    xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->values.size()));
+    std::vector<std::string> ::const_iterator _iter126;
+    for (_iter126 = this->values.begin(); _iter126 != this->values.end(); ++_iter126)
+    {
+      xfer += oprot->writeString((*_iter126));
+    }
+    xfer += oprot->writeListEnd();
+  }
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("nulls", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeBinary(this->nulls);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+void swap(TStringColumn &a, TStringColumn &b) {
+  using ::std::swap;
+  swap(a.values, b.values);
+  swap(a.nulls, b.nulls);
+}
+
+const char* TBinaryColumn::ascii_fingerprint = "BE556BF7091B2DABBA1863D5E458B15F";
+const uint8_t TBinaryColumn::binary_fingerprint[16] = {0xBE,0x55,0x6B,0xF7,0x09,0x1B,0x2D,0xAB,0xBA,0x18,0x63,0xD5,0xE4,0x58,0xB1,0x5F};
+
+uint32_t TBinaryColumn::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+  bool isset_values = false;
+  bool isset_nulls = false;
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_LIST) {
+          {
+            this->values.clear();
+            uint32_t _size127;
+            ::apache::thrift::protocol::TType _etype130;
+            xfer += iprot->readListBegin(_etype130, _size127);
+            this->values.resize(_size127);
+            uint32_t _i131;
+            for (_i131 = 0; _i131 < _size127; ++_i131)
+            {
+              xfer += iprot->readBinary(this->values[_i131]);
+            }
+            xfer += iprot->readListEnd();
+          }
+          isset_values = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_STRING) {
+          xfer += iprot->readBinary(this->nulls);
+          isset_nulls = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  if (!isset_values)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  if (!isset_nulls)
+    throw TProtocolException(TProtocolException::INVALID_DATA);
+  return xfer;
+}
+
+uint32_t TBinaryColumn::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  xfer += oprot->writeStructBegin("TBinaryColumn");
+
+  xfer += oprot->writeFieldBegin("values", ::apache::thrift::protocol::T_LIST, 1);
+  {
+    xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->values.size()));
+    std::vector<std::string> ::const_iterator _iter132;
+    for (_iter132 = this->values.begin(); _iter132 != this->values.end(); ++_iter132)
+    {
+      xfer += oprot->writeBinary((*_iter132));
+    }
+    xfer += oprot->writeListEnd();
+  }
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("nulls", ::apache::thrift::protocol::T_STRING, 2);
+  xfer += oprot->writeBinary(this->nulls);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+void swap(TBinaryColumn &a, TBinaryColumn &b) {
+  using ::std::swap;
+  swap(a.values, b.values);
+  swap(a.nulls, b.nulls);
+}
+
+const char* TColumn::ascii_fingerprint = "E6ADD10B4CDDE61A19E8878CC7039A17";
+const uint8_t TColumn::binary_fingerprint[16] = {0xE6,0xAD,0xD1,0x0B,0x4C,0xDD,0xE6,0x1A,0x19,0xE8,0x87,0x8C,0xC7,0x03,0x9A,0x17};
+
+uint32_t TColumn::read(::apache::thrift::protocol::TProtocol* iprot) {
+
+  uint32_t xfer = 0;
+  std::string fname;
+  ::apache::thrift::protocol::TType ftype;
+  int16_t fid;
+
+  xfer += iprot->readStructBegin(fname);
+
+  using ::apache::thrift::protocol::TProtocolException;
+
+
+  while (true)
+  {
+    xfer += iprot->readFieldBegin(fname, ftype, fid);
+    if (ftype == ::apache::thrift::protocol::T_STOP) {
+      break;
+    }
+    switch (fid)
+    {
+      case 1:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->boolVal.read(iprot);
+          this->__isset.boolVal = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 2:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->byteVal.read(iprot);
+          this->__isset.byteVal = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 3:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->i16Val.read(iprot);
+          this->__isset.i16Val = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 4:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->i32Val.read(iprot);
+          this->__isset.i32Val = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 5:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->i64Val.read(iprot);
+          this->__isset.i64Val = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 6:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->doubleVal.read(iprot);
+          this->__isset.doubleVal = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 7:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->stringVal.read(iprot);
+          this->__isset.stringVal = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      case 8:
+        if (ftype == ::apache::thrift::protocol::T_STRUCT) {
+          xfer += this->binaryVal.read(iprot);
+          this->__isset.binaryVal = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
+      default:
+        xfer += iprot->skip(ftype);
+        break;
+    }
+    xfer += iprot->readFieldEnd();
+  }
+
+  xfer += iprot->readStructEnd();
+
+  return xfer;
+}
+
+uint32_t TColumn::write(::apache::thrift::protocol::TProtocol* oprot) const {
+  uint32_t xfer = 0;
+  xfer += oprot->writeStructBegin("TColumn");
+
+  xfer += oprot->writeFieldBegin("boolVal", ::apache::thrift::protocol::T_STRUCT, 1);
+  xfer += this->boolVal.write(oprot);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("byteVal", ::apache::thrift::protocol::T_STRUCT, 2);
+  xfer += this->byteVal.write(oprot);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("i16Val", ::apache::thrift::protocol::T_STRUCT, 3);
+  xfer += this->i16Val.write(oprot);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("i32Val", ::apache::thrift::protocol::T_STRUCT, 4);
+  xfer += this->i32Val.write(oprot);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("i64Val", ::apache::thrift::protocol::T_STRUCT, 5);
+  xfer += this->i64Val.write(oprot);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("doubleVal", ::apache::thrift::protocol::T_STRUCT, 6);
+  xfer += this->doubleVal.write(oprot);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("stringVal", ::apache::thrift::protocol::T_STRUCT, 7);
+  xfer += this->stringVal.write(oprot);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldBegin("binaryVal", ::apache::thrift::protocol::T_STRUCT, 8);
+  xfer += this->binaryVal.write(oprot);
+  xfer += oprot->writeFieldEnd();
+
+  xfer += oprot->writeFieldStop();
+  xfer += oprot->writeStructEnd();
+  return xfer;
+}
+
+void swap(TColumn &a, TColumn &b) {
+  using ::std::swap;
+  swap(a.boolVal, b.boolVal);
+  swap(a.byteVal, b.byteVal);
+  swap(a.i16Val, b.i16Val);
+  swap(a.i32Val, b.i32Val);
+  swap(a.i64Val, b.i64Val);
+  swap(a.doubleVal, b.doubleVal);
+  swap(a.stringVal, b.stringVal);
+  swap(a.binaryVal, b.binaryVal);
+  swap(a.__isset, b.__isset);
+}
+
+const char* TRowSet::ascii_fingerprint = "46DA30A870489C7A58105AE0080DAEBF";
+const uint8_t TRowSet::binary_fingerprint[16] = {0x46,0xDA,0x30,0xA8,0x70,0x48,0x9C,0x7A,0x58,0x10,0x5A,0xE0,0x08,0x0D,0xAE,0xBF};
 
 uint32_t TRowSet::read(::apache::thrift::protocol::TProtocol* iprot) {
 
@@ -2270,14 +3225,14 @@ uint32_t TRowSet::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->rows.clear();
-            uint32_t _size85;
-            ::apache::thrift::protocol::TType _etype88;
-            xfer += iprot->readListBegin(_etype88, _size85);
-            this->rows.resize(_size85);
-            uint32_t _i89;
-            for (_i89 = 0; _i89 < _size85; ++_i89)
+            uint32_t _size133;
+            ::apache::thrift::protocol::TType _etype136;
+            xfer += iprot->readListBegin(_etype136, _size133);
+            this->rows.resize(_size133);
+            uint32_t _i137;
+            for (_i137 = 0; _i137 < _size133; ++_i137)
             {
-              xfer += this->rows[_i89].read(iprot);
+              xfer += this->rows[_i137].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -2290,14 +3245,14 @@ uint32_t TRowSet::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->columns.clear();
-            uint32_t _size90;
-            ::apache::thrift::protocol::TType _etype93;
-            xfer += iprot->readListBegin(_etype93, _size90);
-            this->columns.resize(_size90);
-            uint32_t _i94;
-            for (_i94 = 0; _i94 < _size90; ++_i94)
+            uint32_t _size138;
+            ::apache::thrift::protocol::TType _etype141;
+            xfer += iprot->readListBegin(_etype141, _size138);
+            this->columns.resize(_size138);
+            uint32_t _i142;
+            for (_i142 = 0; _i142 < _size138; ++_i142)
             {
-              xfer += this->columns[_i94].read(iprot);
+              xfer += this->columns[_i142].read(iprot);
             }
             xfer += iprot->readListEnd();
           }
@@ -2333,10 +3288,10 @@ uint32_t TRowSet::write(::apache::thrift
   xfer += oprot->writeFieldBegin("rows", ::apache::thrift::protocol::T_LIST, 2);
   {
     xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->rows.size()));
-    std::vector<TRow> ::const_iterator _iter95;
-    for (_iter95 = this->rows.begin(); _iter95 != this->rows.end(); ++_iter95)
+    std::vector<TRow> ::const_iterator _iter143;
+    for (_iter143 = this->rows.begin(); _iter143 != this->rows.end(); ++_iter143)
     {
-      xfer += (*_iter95).write(oprot);
+      xfer += (*_iter143).write(oprot);
     }
     xfer += oprot->writeListEnd();
   }
@@ -2346,10 +3301,10 @@ uint32_t TRowSet::write(::apache::thrift
     xfer += oprot->writeFieldBegin("columns", ::apache::thrift::protocol::T_LIST, 3);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast<uint32_t>(this->columns.size()));
-      std::vector<TColumn> ::const_iterator _iter96;
-      for (_iter96 = this->columns.begin(); _iter96 != this->columns.end(); ++_iter96)
+      std::vector<TColumn> ::const_iterator _iter144;
+      for (_iter144 = this->columns.begin(); _iter144 != this->columns.end(); ++_iter144)
       {
-        xfer += (*_iter96).write(oprot);
+        xfer += (*_iter144).write(oprot);
       }
       xfer += oprot->writeListEnd();
     }
@@ -2394,9 +3349,9 @@ uint32_t TStatus::read(::apache::thrift:
     {
       case 1:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast97;
-          xfer += iprot->readI32(ecast97);
-          this->statusCode = (TStatusCode::type)ecast97;
+          int32_t ecast145;
+          xfer += iprot->readI32(ecast145);
+          this->statusCode = (TStatusCode::type)ecast145;
           isset_statusCode = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -2406,14 +3361,14 @@ uint32_t TStatus::read(::apache::thrift:
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->infoMessages.clear();
-            uint32_t _size98;
-            ::apache::thrift::protocol::TType _etype101;
-            xfer += iprot->readListBegin(_etype101, _size98);
-            this->infoMessages.resize(_size98);
-            uint32_t _i102;
-            for (_i102 = 0; _i102 < _size98; ++_i102)
+            uint32_t _size146;
+            ::apache::thrift::protocol::TType _etype149;
+            xfer += iprot->readListBegin(_etype149, _size146);
+            this->infoMessages.resize(_size146);
+            uint32_t _i150;
+            for (_i150 = 0; _i150 < _size146; ++_i150)
             {
-              xfer += iprot->readString(this->infoMessages[_i102]);
+              xfer += iprot->readString(this->infoMessages[_i150]);
             }
             xfer += iprot->readListEnd();
           }
@@ -2472,10 +3427,10 @@ uint32_t TStatus::write(::apache::thrift
     xfer += oprot->writeFieldBegin("infoMessages", ::apache::thrift::protocol::T_LIST, 2);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->infoMessages.size()));
-      std::vector<std::string> ::const_iterator _iter103;
-      for (_iter103 = this->infoMessages.begin(); _iter103 != this->infoMessages.end(); ++_iter103)
+      std::vector<std::string> ::const_iterator _iter151;
+      for (_iter151 = this->infoMessages.begin(); _iter151 != this->infoMessages.end(); ++_iter151)
       {
-        xfer += oprot->writeString((*_iter103));
+        xfer += oprot->writeString((*_iter151));
       }
       xfer += oprot->writeListEnd();
     }
@@ -2691,9 +3646,9 @@ uint32_t TOperationHandle::read(::apache
         break;
       case 2:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast104;
-          xfer += iprot->readI32(ecast104);
-          this->operationType = (TOperationType::type)ecast104;
+          int32_t ecast152;
+          xfer += iprot->readI32(ecast152);
+          this->operationType = (TOperationType::type)ecast152;
           isset_operationType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -2794,9 +3749,9 @@ uint32_t TOpenSessionReq::read(::apache:
     {
       case 1:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast105;
-          xfer += iprot->readI32(ecast105);
-          this->client_protocol = (TProtocolVersion::type)ecast105;
+          int32_t ecast153;
+          xfer += iprot->readI32(ecast153);
+          this->client_protocol = (TProtocolVersion::type)ecast153;
           isset_client_protocol = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -2822,17 +3777,17 @@ uint32_t TOpenSessionReq::read(::apache:
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->configuration.clear();
-            uint32_t _size106;
-            ::apache::thrift::protocol::TType _ktype107;
-            ::apache::thrift::protocol::TType _vtype108;
-            xfer += iprot->readMapBegin(_ktype107, _vtype108, _size106);
-            uint32_t _i110;
-            for (_i110 = 0; _i110 < _size106; ++_i110)
+            uint32_t _size154;
+            ::apache::thrift::protocol::TType _ktype155;
+            ::apache::thrift::protocol::TType _vtype156;
+            xfer += iprot->readMapBegin(_ktype155, _vtype156, _size154);
+            uint32_t _i158;
+            for (_i158 = 0; _i158 < _size154; ++_i158)
             {
-              std::string _key111;
-              xfer += iprot->readString(_key111);
-              std::string& _val112 = this->configuration[_key111];
-              xfer += iprot->readString(_val112);
+              std::string _key159;
+              xfer += iprot->readString(_key159);
+              std::string& _val160 = this->configuration[_key159];
+              xfer += iprot->readString(_val160);
             }
             xfer += iprot->readMapEnd();
           }
@@ -2877,11 +3832,11 @@ uint32_t TOpenSessionReq::write(::apache
     xfer += oprot->writeFieldBegin("configuration", ::apache::thrift::protocol::T_MAP, 4);
     {
       xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->configuration.size()));
-      std::map<std::string, std::string> ::const_iterator _iter113;
-      for (_iter113 = this->configuration.begin(); _iter113 != this->configuration.end(); ++_iter113)
+      std::map<std::string, std::string> ::const_iterator _iter161;
+      for (_iter161 = this->configuration.begin(); _iter161 != this->configuration.end(); ++_iter161)
       {
-        xfer += oprot->writeString(_iter113->first);
-        xfer += oprot->writeString(_iter113->second);
+        xfer += oprot->writeString(_iter161->first);
+        xfer += oprot->writeString(_iter161->second);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -2936,9 +3891,9 @@ uint32_t TOpenSessionResp::read(::apache
         break;
       case 2:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast114;
-          xfer += iprot->readI32(ecast114);
-          this->serverProtocolVersion = (TProtocolVersion::type)ecast114;
+          int32_t ecast162;
+          xfer += iprot->readI32(ecast162);
+          this->serverProtocolVersion = (TProtocolVersion::type)ecast162;
           isset_serverProtocolVersion = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -2956,17 +3911,17 @@ uint32_t TOpenSessionResp::read(::apache
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->configuration.clear();
-            uint32_t _size115;
-            ::apache::thrift::protocol::TType _ktype116;
-            ::apache::thrift::protocol::TType _vtype117;
-            xfer += iprot->readMapBegin(_ktype116, _vtype117, _size115);
-            uint32_t _i119;
-            for (_i119 = 0; _i119 < _size115; ++_i119)
+            uint32_t _size163;
+            ::apache::thrift::protocol::TType _ktype164;
+            ::apache::thrift::protocol::TType _vtype165;
+            xfer += iprot->readMapBegin(_ktype164, _vtype165, _size163);
+            uint32_t _i167;
+            for (_i167 = 0; _i167 < _size163; ++_i167)
             {
-              std::string _key120;
-              xfer += iprot->readString(_key120);
-              std::string& _val121 = this->configuration[_key120];
-              xfer += iprot->readString(_val121);
+              std::string _key168;
+              xfer += iprot->readString(_key168);
+              std::string& _val169 = this->configuration[_key168];
+              xfer += iprot->readString(_val169);
             }
             xfer += iprot->readMapEnd();
           }
@@ -3012,11 +3967,11 @@ uint32_t TOpenSessionResp::write(::apach
     xfer += oprot->writeFieldBegin("configuration", ::apache::thrift::protocol::T_MAP, 4);
     {
       xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->configuration.size()));
-      std::map<std::string, std::string> ::const_iterator _iter122;
-      for (_iter122 = this->configuration.begin(); _iter122 != this->configuration.end(); ++_iter122)
+      std::map<std::string, std::string> ::const_iterator _iter170;
+      for (_iter170 = this->configuration.begin(); _iter170 != this->configuration.end(); ++_iter170)
       {
-        xfer += oprot->writeString(_iter122->first);
-        xfer += oprot->writeString(_iter122->second);
+        xfer += oprot->writeString(_iter170->first);
+        xfer += oprot->writeString(_iter170->second);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -3326,9 +4281,9 @@ uint32_t TGetInfoReq::read(::apache::thr
         break;
       case 2:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast123;
-          xfer += iprot->readI32(ecast123);
-          this->infoType = (TGetInfoType::type)ecast123;
+          int32_t ecast171;
+          xfer += iprot->readI32(ecast171);
+          this->infoType = (TGetInfoType::type)ecast171;
           isset_infoType = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -3498,17 +4453,17 @@ uint32_t TExecuteStatementReq::read(::ap
         if (ftype == ::apache::thrift::protocol::T_MAP) {
           {
             this->confOverlay.clear();
-            uint32_t _size124;
-            ::apache::thrift::protocol::TType _ktype125;
-            ::apache::thrift::protocol::TType _vtype126;
-            xfer += iprot->readMapBegin(_ktype125, _vtype126, _size124);
-            uint32_t _i128;
-            for (_i128 = 0; _i128 < _size124; ++_i128)
+            uint32_t _size172;
+            ::apache::thrift::protocol::TType _ktype173;
+            ::apache::thrift::protocol::TType _vtype174;
+            xfer += iprot->readMapBegin(_ktype173, _vtype174, _size172);
+            uint32_t _i176;
+            for (_i176 = 0; _i176 < _size172; ++_i176)
             {
-              std::string _key129;
-              xfer += iprot->readString(_key129);
-              std::string& _val130 = this->confOverlay[_key129];
-              xfer += iprot->readString(_val130);
+              std::string _key177;
+              xfer += iprot->readString(_key177);
+              std::string& _val178 = this->confOverlay[_key177];
+              xfer += iprot->readString(_val178);
             }
             xfer += iprot->readMapEnd();
           }
@@ -3557,11 +4512,11 @@ uint32_t TExecuteStatementReq::write(::a
     xfer += oprot->writeFieldBegin("confOverlay", ::apache::thrift::protocol::T_MAP, 3);
     {
       xfer += oprot->writeMapBegin(::apache::thrift::protocol::T_STRING, ::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->confOverlay.size()));
-      std::map<std::string, std::string> ::const_iterator _iter131;
-      for (_iter131 = this->confOverlay.begin(); _iter131 != this->confOverlay.end(); ++_iter131)
+      std::map<std::string, std::string> ::const_iterator _iter179;
+      for (_iter179 = this->confOverlay.begin(); _iter179 != this->confOverlay.end(); ++_iter179)
       {
-        xfer += oprot->writeString(_iter131->first);
-        xfer += oprot->writeString(_iter131->second);
+        xfer += oprot->writeString(_iter179->first);
+        xfer += oprot->writeString(_iter179->second);
       }
       xfer += oprot->writeMapEnd();
     }
@@ -4183,14 +5138,14 @@ uint32_t TGetTablesReq::read(::apache::t
         if (ftype == ::apache::thrift::protocol::T_LIST) {
           {
             this->tableTypes.clear();
-            uint32_t _size132;
-            ::apache::thrift::protocol::TType _etype135;
-            xfer += iprot->readListBegin(_etype135, _size132);
-            this->tableTypes.resize(_size132);
-            uint32_t _i136;
-            for (_i136 = 0; _i136 < _size132; ++_i136)
+            uint32_t _size180;
+            ::apache::thrift::protocol::TType _etype183;
+            xfer += iprot->readListBegin(_etype183, _size180);
+            this->tableTypes.resize(_size180);
+            uint32_t _i184;
+            for (_i184 = 0; _i184 < _size180; ++_i184)
             {
-              xfer += iprot->readString(this->tableTypes[_i136]);
+              xfer += iprot->readString(this->tableTypes[_i184]);
             }
             xfer += iprot->readListEnd();
           }
@@ -4240,10 +5195,10 @@ uint32_t TGetTablesReq::write(::apache::
     xfer += oprot->writeFieldBegin("tableTypes", ::apache::thrift::protocol::T_LIST, 5);
     {
       xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast<uint32_t>(this->tableTypes.size()));
-      std::vector<std::string> ::const_iterator _iter137;
-      for (_iter137 = this->tableTypes.begin(); _iter137 != this->tableTypes.end(); ++_iter137)
+      std::vector<std::string> ::const_iterator _iter185;
+      for (_iter185 = this->tableTypes.begin(); _iter185 != this->tableTypes.end(); ++_iter185)
       {
-        xfer += oprot->writeString((*_iter137));
+        xfer += oprot->writeString((*_iter185));
       }
       xfer += oprot->writeListEnd();
     }
@@ -4972,9 +5927,9 @@ uint32_t TGetOperationStatusResp::read(:
         break;
       case 2:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast138;
-          xfer += iprot->readI32(ecast138);
-          this->operationState = (TOperationState::type)ecast138;
+          int32_t ecast186;
+          xfer += iprot->readI32(ecast186);
+          this->operationState = (TOperationState::type)ecast186;
           this->__isset.operationState = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -5496,9 +6451,9 @@ uint32_t TFetchResultsReq::read(::apache
         break;
       case 2:
         if (ftype == ::apache::thrift::protocol::T_I32) {
-          int32_t ecast139;
-          xfer += iprot->readI32(ecast139);
-          this->orientation = (TFetchOrientation::type)ecast139;
+          int32_t ecast187;
+          xfer += iprot->readI32(ecast187);
+          this->orientation = (TFetchOrientation::type)ecast187;
           isset_orientation = true;
         } else {
           xfer += iprot->skip(ftype);
@@ -5558,8 +6513,8 @@ void swap(TFetchResultsReq &a, TFetchRes
   swap(a.maxRows, b.maxRows);
 }
 
-const char* TFetchResultsResp::ascii_fingerprint = "29891EA4D71B4283E8715DA5B95F2763";
-const uint8_t TFetchResultsResp::binary_fingerprint[16] = {0x29,0x89,0x1E,0xA4,0xD7,0x1B,0x42,0x83,0xE8,0x71,0x5D,0xA5,0xB9,0x5F,0x27,0x63};
+const char* TFetchResultsResp::ascii_fingerprint = "FC43BC2D6F3B76D4DB0F34226A745C8E";
+const uint8_t TFetchResultsResp::binary_fingerprint[16] = {0xFC,0x43,0xBC,0x2D,0x6F,0x3B,0x76,0xD4,0xDB,0x0F,0x34,0x22,0x6A,0x74,0x5C,0x8E};
 
 uint32_t TFetchResultsResp::read(::apache::thrift::protocol::TProtocol* iprot) {