diff --git common/src/java/org/apache/hadoop/hive/conf/HiveConf.java common/src/java/org/apache/hadoop/hive/conf/HiveConf.java index 27f68df..0f595bc 100644 --- common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -2195,8 +2195,14 @@ public void setSparkConfigUpdated(boolean isSparkConfigUpdated) { new StringSet("throw", "skip", "ignore"), "The approach msck should take with HDFS " + "directories that are partition-like but contain unsupported characters. 'throw' (an " + "exception) is the default; 'skip' will skip the invalid directories and still repair the" + - " others; 'ignore' will skip the validation (legacy behavior, causes bugs in many cases)"); - + " others; 'ignore' will skip the validation (legacy behavior, causes bugs in many cases)"), + + // Hiveserver2 resultset compression configurations + HIVE_RESULTSET_COMPRESSION_DISABLED_COMPRESSORS("hive.resultset.compression.disabled.compressors", "", + "Set a list of disabled resultset compressors"), + HIVE_RESULTSET_COMPRESSION_ENABLED("hive.resultset.compression.enabled", true, + "Whether to enable resultset compression"), + ; public final String varname; private final String defaultExpr; diff --git conf/hive-site.xml conf/hive-site.xml index dab494e..58de017 100644 --- conf/hive-site.xml +++ conf/hive-site.xml @@ -18,5 +18,15 @@ --> - + + hive.resultset.compression.disabled.compressors + + + ResultSet compression sets that shouldn't be used for any reason, should be mentioned here as comma separated values. + + + + hive.resultset.compression.enabled + true + diff --git service/if/TCLIService.thrift service/if/TCLIService.thrift index baf583f..8764e5c 100644 --- service/if/TCLIService.thrift +++ service/if/TCLIService.thrift @@ -60,6 +60,9 @@ enum TProtocolVersion { // V8 adds support for interval types HIVE_CLI_SERVICE_PROTOCOL_V8 + + // V9 adds support for encoding (compressing) columnar result set + HIVE_CLI_SERVICE_PROTOCOL_V9 } enum TTypeId { @@ -396,12 +399,23 @@ union TColumn { 8: TBinaryColumn binaryVal // BINARY } +//Represents an encoded column +struct TEnColumn { + 1: required binary enData + 2: required binary nulls + 3: required TTypeId type + 4: required i32 size + 5: required string compressorName +} + // Represents a rowset struct TRowSet { // The starting row offset of this rowset. 1: required i64 startRowOffset 2: required list rows 3: optional list columns + 4: optional list enColumns + 5: optional binary compressorBitmap } // The return status code contained in each response. diff --git service/src/gen/thrift/gen-cpp/TCLIService_types.cpp service/src/gen/thrift/gen-cpp/TCLIService_types.cpp index 86eeea3..2b9d661 100644 --- service/src/gen/thrift/gen-cpp/TCLIService_types.cpp +++ service/src/gen/thrift/gen-cpp/TCLIService_types.cpp @@ -18,7 +18,8 @@ int _kTProtocolVersionValues[] = { TProtocolVersion::HIVE_CLI_SERVICE_PROTOCOL_V5, TProtocolVersion::HIVE_CLI_SERVICE_PROTOCOL_V6, TProtocolVersion::HIVE_CLI_SERVICE_PROTOCOL_V7, - TProtocolVersion::HIVE_CLI_SERVICE_PROTOCOL_V8 + TProtocolVersion::HIVE_CLI_SERVICE_PROTOCOL_V8, + TProtocolVersion::HIVE_CLI_SERVICE_PROTOCOL_V9 }; const char* _kTProtocolVersionNames[] = { "HIVE_CLI_SERVICE_PROTOCOL_V1", @@ -28,9 +29,10 @@ const char* _kTProtocolVersionNames[] = { "HIVE_CLI_SERVICE_PROTOCOL_V5", "HIVE_CLI_SERVICE_PROTOCOL_V6", "HIVE_CLI_SERVICE_PROTOCOL_V7", - "HIVE_CLI_SERVICE_PROTOCOL_V8" + "HIVE_CLI_SERVICE_PROTOCOL_V8", + "HIVE_CLI_SERVICE_PROTOCOL_V9" }; -const std::map _TProtocolVersion_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(8, _kTProtocolVersionValues, _kTProtocolVersionNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); +const std::map _TProtocolVersion_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(9, _kTProtocolVersionValues, _kTProtocolVersionNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); int _kTTypeIdValues[] = { TTypeId::BOOLEAN_TYPE, @@ -2916,8 +2918,138 @@ void swap(TColumn &a, TColumn &b) { 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}; +const char* TEnColumn::ascii_fingerprint = "C4125B153F648260127E2F8907DB448A"; +const uint8_t TEnColumn::binary_fingerprint[16] = {0xC4,0x12,0x5B,0x15,0x3F,0x64,0x82,0x60,0x12,0x7E,0x2F,0x89,0x07,0xDB,0x44,0x8A}; + +uint32_t TEnColumn::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_enData = false; + bool isset_nulls = false; + bool isset_type = false; + bool isset_size = false; + bool isset_compressorName = 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_STRING) { + xfer += iprot->readBinary(this->enData); + isset_enData = 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; + case 3: + if (ftype == ::apache::thrift::protocol::T_I32) { + int32_t ecast91; + xfer += iprot->readI32(ecast91); + this->type = (TTypeId::type)ecast91; + isset_type = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 4: + if (ftype == ::apache::thrift::protocol::T_I32) { + xfer += iprot->readI32(this->size); + isset_size = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readString(this->compressorName); + isset_compressorName = true; + } else { + xfer += iprot->skip(ftype); + } + break; + default: + xfer += iprot->skip(ftype); + break; + } + xfer += iprot->readFieldEnd(); + } + + xfer += iprot->readStructEnd(); + + if (!isset_enData) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_nulls) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_type) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_size) + throw TProtocolException(TProtocolException::INVALID_DATA); + if (!isset_compressorName) + throw TProtocolException(TProtocolException::INVALID_DATA); + return xfer; +} + +uint32_t TEnColumn::write(::apache::thrift::protocol::TProtocol* oprot) const { + uint32_t xfer = 0; + xfer += oprot->writeStructBegin("TEnColumn"); + + xfer += oprot->writeFieldBegin("enData", ::apache::thrift::protocol::T_STRING, 1); + xfer += oprot->writeBinary(this->enData); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("nulls", ::apache::thrift::protocol::T_STRING, 2); + xfer += oprot->writeBinary(this->nulls); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("type", ::apache::thrift::protocol::T_I32, 3); + xfer += oprot->writeI32((int32_t)this->type); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("size", ::apache::thrift::protocol::T_I32, 4); + xfer += oprot->writeI32(this->size); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldBegin("compressorName", ::apache::thrift::protocol::T_STRING, 5); + xfer += oprot->writeString(this->compressorName); + xfer += oprot->writeFieldEnd(); + + xfer += oprot->writeFieldStop(); + xfer += oprot->writeStructEnd(); + return xfer; +} + +void swap(TEnColumn &a, TEnColumn &b) { + using ::std::swap; + swap(a.enData, b.enData); + swap(a.nulls, b.nulls); + swap(a.type, b.type); + swap(a.size, b.size); + swap(a.compressorName, b.compressorName); +} + +const char* TRowSet::ascii_fingerprint = "8E980BCEDB71FA696E49ECA856CDDB6F"; +const uint8_t TRowSet::binary_fingerprint[16] = {0x8E,0x98,0x0B,0xCE,0xDB,0x71,0xFA,0x69,0x6E,0x49,0xEC,0xA8,0x56,0xCD,0xDB,0x6F}; uint32_t TRowSet::read(::apache::thrift::protocol::TProtocol* iprot) { @@ -2953,14 +3085,14 @@ uint32_t TRowSet::read(::apache::thrift::protocol::TProtocol* iprot) { if (ftype == ::apache::thrift::protocol::T_LIST) { { this->rows.clear(); - uint32_t _size91; - ::apache::thrift::protocol::TType _etype94; - xfer += iprot->readListBegin(_etype94, _size91); - this->rows.resize(_size91); - uint32_t _i95; - for (_i95 = 0; _i95 < _size91; ++_i95) + uint32_t _size92; + ::apache::thrift::protocol::TType _etype95; + xfer += iprot->readListBegin(_etype95, _size92); + this->rows.resize(_size92); + uint32_t _i96; + for (_i96 = 0; _i96 < _size92; ++_i96) { - xfer += this->rows[_i95].read(iprot); + xfer += this->rows[_i96].read(iprot); } xfer += iprot->readListEnd(); } @@ -2973,14 +3105,14 @@ uint32_t TRowSet::read(::apache::thrift::protocol::TProtocol* iprot) { if (ftype == ::apache::thrift::protocol::T_LIST) { { this->columns.clear(); - uint32_t _size96; - ::apache::thrift::protocol::TType _etype99; - xfer += iprot->readListBegin(_etype99, _size96); - this->columns.resize(_size96); - uint32_t _i100; - for (_i100 = 0; _i100 < _size96; ++_i100) + uint32_t _size97; + ::apache::thrift::protocol::TType _etype100; + xfer += iprot->readListBegin(_etype100, _size97); + this->columns.resize(_size97); + uint32_t _i101; + for (_i101 = 0; _i101 < _size97; ++_i101) { - xfer += this->columns[_i100].read(iprot); + xfer += this->columns[_i101].read(iprot); } xfer += iprot->readListEnd(); } @@ -2989,6 +3121,34 @@ uint32_t TRowSet::read(::apache::thrift::protocol::TProtocol* iprot) { xfer += iprot->skip(ftype); } break; + case 4: + if (ftype == ::apache::thrift::protocol::T_LIST) { + { + this->enColumns.clear(); + uint32_t _size102; + ::apache::thrift::protocol::TType _etype105; + xfer += iprot->readListBegin(_etype105, _size102); + this->enColumns.resize(_size102); + uint32_t _i106; + for (_i106 = 0; _i106 < _size102; ++_i106) + { + xfer += this->enColumns[_i106].read(iprot); + } + xfer += iprot->readListEnd(); + } + this->__isset.enColumns = true; + } else { + xfer += iprot->skip(ftype); + } + break; + case 5: + if (ftype == ::apache::thrift::protocol::T_STRING) { + xfer += iprot->readBinary(this->compressorBitmap); + this->__isset.compressorBitmap = true; + } else { + xfer += iprot->skip(ftype); + } + break; default: xfer += iprot->skip(ftype); break; @@ -3016,10 +3176,10 @@ uint32_t TRowSet::write(::apache::thrift::protocol::TProtocol* oprot) const { xfer += oprot->writeFieldBegin("rows", ::apache::thrift::protocol::T_LIST, 2); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->rows.size())); - std::vector ::const_iterator _iter101; - for (_iter101 = this->rows.begin(); _iter101 != this->rows.end(); ++_iter101) + std::vector ::const_iterator _iter107; + for (_iter107 = this->rows.begin(); _iter107 != this->rows.end(); ++_iter107) { - xfer += (*_iter101).write(oprot); + xfer += (*_iter107).write(oprot); } xfer += oprot->writeListEnd(); } @@ -3029,15 +3189,33 @@ uint32_t TRowSet::write(::apache::thrift::protocol::TProtocol* oprot) const { xfer += oprot->writeFieldBegin("columns", ::apache::thrift::protocol::T_LIST, 3); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->columns.size())); - std::vector ::const_iterator _iter102; - for (_iter102 = this->columns.begin(); _iter102 != this->columns.end(); ++_iter102) + std::vector ::const_iterator _iter108; + for (_iter108 = this->columns.begin(); _iter108 != this->columns.end(); ++_iter108) + { + xfer += (*_iter108).write(oprot); + } + xfer += oprot->writeListEnd(); + } + xfer += oprot->writeFieldEnd(); + } + if (this->__isset.enColumns) { + xfer += oprot->writeFieldBegin("enColumns", ::apache::thrift::protocol::T_LIST, 4); + { + xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->enColumns.size())); + std::vector ::const_iterator _iter109; + for (_iter109 = this->enColumns.begin(); _iter109 != this->enColumns.end(); ++_iter109) { - xfer += (*_iter102).write(oprot); + xfer += (*_iter109).write(oprot); } xfer += oprot->writeListEnd(); } xfer += oprot->writeFieldEnd(); } + if (this->__isset.compressorBitmap) { + xfer += oprot->writeFieldBegin("compressorBitmap", ::apache::thrift::protocol::T_STRING, 5); + xfer += oprot->writeBinary(this->compressorBitmap); + xfer += oprot->writeFieldEnd(); + } xfer += oprot->writeFieldStop(); xfer += oprot->writeStructEnd(); return xfer; @@ -3048,6 +3226,8 @@ void swap(TRowSet &a, TRowSet &b) { swap(a.startRowOffset, b.startRowOffset); swap(a.rows, b.rows); swap(a.columns, b.columns); + swap(a.enColumns, b.enColumns); + swap(a.compressorBitmap, b.compressorBitmap); swap(a.__isset, b.__isset); } @@ -3077,9 +3257,9 @@ uint32_t TStatus::read(::apache::thrift::protocol::TProtocol* iprot) { { case 1: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast103; - xfer += iprot->readI32(ecast103); - this->statusCode = (TStatusCode::type)ecast103; + int32_t ecast110; + xfer += iprot->readI32(ecast110); + this->statusCode = (TStatusCode::type)ecast110; isset_statusCode = true; } else { xfer += iprot->skip(ftype); @@ -3089,14 +3269,14 @@ uint32_t TStatus::read(::apache::thrift::protocol::TProtocol* iprot) { if (ftype == ::apache::thrift::protocol::T_LIST) { { this->infoMessages.clear(); - uint32_t _size104; - ::apache::thrift::protocol::TType _etype107; - xfer += iprot->readListBegin(_etype107, _size104); - this->infoMessages.resize(_size104); - uint32_t _i108; - for (_i108 = 0; _i108 < _size104; ++_i108) + uint32_t _size111; + ::apache::thrift::protocol::TType _etype114; + xfer += iprot->readListBegin(_etype114, _size111); + this->infoMessages.resize(_size111); + uint32_t _i115; + for (_i115 = 0; _i115 < _size111; ++_i115) { - xfer += iprot->readString(this->infoMessages[_i108]); + xfer += iprot->readString(this->infoMessages[_i115]); } xfer += iprot->readListEnd(); } @@ -3155,10 +3335,10 @@ uint32_t TStatus::write(::apache::thrift::protocol::TProtocol* oprot) const { xfer += oprot->writeFieldBegin("infoMessages", ::apache::thrift::protocol::T_LIST, 2); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->infoMessages.size())); - std::vector ::const_iterator _iter109; - for (_iter109 = this->infoMessages.begin(); _iter109 != this->infoMessages.end(); ++_iter109) + std::vector ::const_iterator _iter116; + for (_iter116 = this->infoMessages.begin(); _iter116 != this->infoMessages.end(); ++_iter116) { - xfer += oprot->writeString((*_iter109)); + xfer += oprot->writeString((*_iter116)); } xfer += oprot->writeListEnd(); } @@ -3374,9 +3554,9 @@ uint32_t TOperationHandle::read(::apache::thrift::protocol::TProtocol* iprot) { break; case 2: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast110; - xfer += iprot->readI32(ecast110); - this->operationType = (TOperationType::type)ecast110; + int32_t ecast117; + xfer += iprot->readI32(ecast117); + this->operationType = (TOperationType::type)ecast117; isset_operationType = true; } else { xfer += iprot->skip(ftype); @@ -3477,9 +3657,9 @@ uint32_t TOpenSessionReq::read(::apache::thrift::protocol::TProtocol* iprot) { { case 1: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast111; - xfer += iprot->readI32(ecast111); - this->client_protocol = (TProtocolVersion::type)ecast111; + int32_t ecast118; + xfer += iprot->readI32(ecast118); + this->client_protocol = (TProtocolVersion::type)ecast118; isset_client_protocol = true; } else { xfer += iprot->skip(ftype); @@ -3505,17 +3685,17 @@ uint32_t TOpenSessionReq::read(::apache::thrift::protocol::TProtocol* iprot) { if (ftype == ::apache::thrift::protocol::T_MAP) { { this->configuration.clear(); - uint32_t _size112; - ::apache::thrift::protocol::TType _ktype113; - ::apache::thrift::protocol::TType _vtype114; - xfer += iprot->readMapBegin(_ktype113, _vtype114, _size112); - uint32_t _i116; - for (_i116 = 0; _i116 < _size112; ++_i116) + uint32_t _size119; + ::apache::thrift::protocol::TType _ktype120; + ::apache::thrift::protocol::TType _vtype121; + xfer += iprot->readMapBegin(_ktype120, _vtype121, _size119); + uint32_t _i123; + for (_i123 = 0; _i123 < _size119; ++_i123) { - std::string _key117; - xfer += iprot->readString(_key117); - std::string& _val118 = this->configuration[_key117]; - xfer += iprot->readString(_val118); + std::string _key124; + xfer += iprot->readString(_key124); + std::string& _val125 = this->configuration[_key124]; + xfer += iprot->readString(_val125); } xfer += iprot->readMapEnd(); } @@ -3560,11 +3740,11 @@ uint32_t TOpenSessionReq::write(::apache::thrift::protocol::TProtocol* oprot) co 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(this->configuration.size())); - std::map ::const_iterator _iter119; - for (_iter119 = this->configuration.begin(); _iter119 != this->configuration.end(); ++_iter119) + std::map ::const_iterator _iter126; + for (_iter126 = this->configuration.begin(); _iter126 != this->configuration.end(); ++_iter126) { - xfer += oprot->writeString(_iter119->first); - xfer += oprot->writeString(_iter119->second); + xfer += oprot->writeString(_iter126->first); + xfer += oprot->writeString(_iter126->second); } xfer += oprot->writeMapEnd(); } @@ -3619,9 +3799,9 @@ uint32_t TOpenSessionResp::read(::apache::thrift::protocol::TProtocol* iprot) { break; case 2: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast120; - xfer += iprot->readI32(ecast120); - this->serverProtocolVersion = (TProtocolVersion::type)ecast120; + int32_t ecast127; + xfer += iprot->readI32(ecast127); + this->serverProtocolVersion = (TProtocolVersion::type)ecast127; isset_serverProtocolVersion = true; } else { xfer += iprot->skip(ftype); @@ -3639,17 +3819,17 @@ uint32_t TOpenSessionResp::read(::apache::thrift::protocol::TProtocol* iprot) { if (ftype == ::apache::thrift::protocol::T_MAP) { { this->configuration.clear(); - uint32_t _size121; - ::apache::thrift::protocol::TType _ktype122; - ::apache::thrift::protocol::TType _vtype123; - xfer += iprot->readMapBegin(_ktype122, _vtype123, _size121); - uint32_t _i125; - for (_i125 = 0; _i125 < _size121; ++_i125) + uint32_t _size128; + ::apache::thrift::protocol::TType _ktype129; + ::apache::thrift::protocol::TType _vtype130; + xfer += iprot->readMapBegin(_ktype129, _vtype130, _size128); + uint32_t _i132; + for (_i132 = 0; _i132 < _size128; ++_i132) { - std::string _key126; - xfer += iprot->readString(_key126); - std::string& _val127 = this->configuration[_key126]; - xfer += iprot->readString(_val127); + std::string _key133; + xfer += iprot->readString(_key133); + std::string& _val134 = this->configuration[_key133]; + xfer += iprot->readString(_val134); } xfer += iprot->readMapEnd(); } @@ -3695,11 +3875,11 @@ uint32_t TOpenSessionResp::write(::apache::thrift::protocol::TProtocol* oprot) c 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(this->configuration.size())); - std::map ::const_iterator _iter128; - for (_iter128 = this->configuration.begin(); _iter128 != this->configuration.end(); ++_iter128) + std::map ::const_iterator _iter135; + for (_iter135 = this->configuration.begin(); _iter135 != this->configuration.end(); ++_iter135) { - xfer += oprot->writeString(_iter128->first); - xfer += oprot->writeString(_iter128->second); + xfer += oprot->writeString(_iter135->first); + xfer += oprot->writeString(_iter135->second); } xfer += oprot->writeMapEnd(); } @@ -4009,9 +4189,9 @@ uint32_t TGetInfoReq::read(::apache::thrift::protocol::TProtocol* iprot) { break; case 2: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast129; - xfer += iprot->readI32(ecast129); - this->infoType = (TGetInfoType::type)ecast129; + int32_t ecast136; + xfer += iprot->readI32(ecast136); + this->infoType = (TGetInfoType::type)ecast136; isset_infoType = true; } else { xfer += iprot->skip(ftype); @@ -4181,17 +4361,17 @@ uint32_t TExecuteStatementReq::read(::apache::thrift::protocol::TProtocol* iprot if (ftype == ::apache::thrift::protocol::T_MAP) { { this->confOverlay.clear(); - uint32_t _size130; - ::apache::thrift::protocol::TType _ktype131; - ::apache::thrift::protocol::TType _vtype132; - xfer += iprot->readMapBegin(_ktype131, _vtype132, _size130); - uint32_t _i134; - for (_i134 = 0; _i134 < _size130; ++_i134) + uint32_t _size137; + ::apache::thrift::protocol::TType _ktype138; + ::apache::thrift::protocol::TType _vtype139; + xfer += iprot->readMapBegin(_ktype138, _vtype139, _size137); + uint32_t _i141; + for (_i141 = 0; _i141 < _size137; ++_i141) { - std::string _key135; - xfer += iprot->readString(_key135); - std::string& _val136 = this->confOverlay[_key135]; - xfer += iprot->readString(_val136); + std::string _key142; + xfer += iprot->readString(_key142); + std::string& _val143 = this->confOverlay[_key142]; + xfer += iprot->readString(_val143); } xfer += iprot->readMapEnd(); } @@ -4240,11 +4420,11 @@ uint32_t TExecuteStatementReq::write(::apache::thrift::protocol::TProtocol* opro 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(this->confOverlay.size())); - std::map ::const_iterator _iter137; - for (_iter137 = this->confOverlay.begin(); _iter137 != this->confOverlay.end(); ++_iter137) + std::map ::const_iterator _iter144; + for (_iter144 = this->confOverlay.begin(); _iter144 != this->confOverlay.end(); ++_iter144) { - xfer += oprot->writeString(_iter137->first); - xfer += oprot->writeString(_iter137->second); + xfer += oprot->writeString(_iter144->first); + xfer += oprot->writeString(_iter144->second); } xfer += oprot->writeMapEnd(); } @@ -4866,14 +5046,14 @@ uint32_t TGetTablesReq::read(::apache::thrift::protocol::TProtocol* iprot) { if (ftype == ::apache::thrift::protocol::T_LIST) { { this->tableTypes.clear(); - uint32_t _size138; - ::apache::thrift::protocol::TType _etype141; - xfer += iprot->readListBegin(_etype141, _size138); - this->tableTypes.resize(_size138); - uint32_t _i142; - for (_i142 = 0; _i142 < _size138; ++_i142) + uint32_t _size145; + ::apache::thrift::protocol::TType _etype148; + xfer += iprot->readListBegin(_etype148, _size145); + this->tableTypes.resize(_size145); + uint32_t _i149; + for (_i149 = 0; _i149 < _size145; ++_i149) { - xfer += iprot->readString(this->tableTypes[_i142]); + xfer += iprot->readString(this->tableTypes[_i149]); } xfer += iprot->readListEnd(); } @@ -4923,10 +5103,10 @@ uint32_t TGetTablesReq::write(::apache::thrift::protocol::TProtocol* oprot) cons xfer += oprot->writeFieldBegin("tableTypes", ::apache::thrift::protocol::T_LIST, 5); { xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->tableTypes.size())); - std::vector ::const_iterator _iter143; - for (_iter143 = this->tableTypes.begin(); _iter143 != this->tableTypes.end(); ++_iter143) + std::vector ::const_iterator _iter150; + for (_iter150 = this->tableTypes.begin(); _iter150 != this->tableTypes.end(); ++_iter150) { - xfer += oprot->writeString((*_iter143)); + xfer += oprot->writeString((*_iter150)); } xfer += oprot->writeListEnd(); } @@ -5655,9 +5835,9 @@ uint32_t TGetOperationStatusResp::read(::apache::thrift::protocol::TProtocol* ip break; case 2: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast144; - xfer += iprot->readI32(ecast144); - this->operationState = (TOperationState::type)ecast144; + int32_t ecast151; + xfer += iprot->readI32(ecast151); + this->operationState = (TOperationState::type)ecast151; this->__isset.operationState = true; } else { xfer += iprot->skip(ftype); @@ -6179,9 +6359,9 @@ uint32_t TFetchResultsReq::read(::apache::thrift::protocol::TProtocol* iprot) { break; case 2: if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast145; - xfer += iprot->readI32(ecast145); - this->orientation = (TFetchOrientation::type)ecast145; + int32_t ecast152; + xfer += iprot->readI32(ecast152); + this->orientation = (TFetchOrientation::type)ecast152; isset_orientation = true; } else { xfer += iprot->skip(ftype); @@ -6256,8 +6436,8 @@ void swap(TFetchResultsReq &a, TFetchResultsReq &b) { swap(a.__isset, b.__isset); } -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}; +const char* TFetchResultsResp::ascii_fingerprint = "99C26CD71A5438BD7A817ACDDDF8C79B"; +const uint8_t TFetchResultsResp::binary_fingerprint[16] = {0x99,0xC2,0x6C,0xD7,0x1A,0x54,0x38,0xBD,0x7A,0x81,0x7A,0xCD,0xDD,0xF8,0xC7,0x9B}; uint32_t TFetchResultsResp::read(::apache::thrift::protocol::TProtocol* iprot) { diff --git service/src/gen/thrift/gen-cpp/TCLIService_types.h service/src/gen/thrift/gen-cpp/TCLIService_types.h index 7bceabd..0e7aa5d 100644 --- service/src/gen/thrift/gen-cpp/TCLIService_types.h +++ service/src/gen/thrift/gen-cpp/TCLIService_types.h @@ -25,7 +25,8 @@ struct TProtocolVersion { HIVE_CLI_SERVICE_PROTOCOL_V5 = 4, HIVE_CLI_SERVICE_PROTOCOL_V6 = 5, HIVE_CLI_SERVICE_PROTOCOL_V7 = 6, - HIVE_CLI_SERVICE_PROTOCOL_V8 = 7 + HIVE_CLI_SERVICE_PROTOCOL_V8 = 7, + HIVE_CLI_SERVICE_PROTOCOL_V9 = 8 }; }; @@ -1642,18 +1643,85 @@ class TColumn { void swap(TColumn &a, TColumn &b); + +class TEnColumn { + public: + + static const char* ascii_fingerprint; // = "C4125B153F648260127E2F8907DB448A"; + static const uint8_t binary_fingerprint[16]; // = {0xC4,0x12,0x5B,0x15,0x3F,0x64,0x82,0x60,0x12,0x7E,0x2F,0x89,0x07,0xDB,0x44,0x8A}; + + TEnColumn() : enData(), nulls(), type((TTypeId::type)0), size(0), compressorName() { + } + + virtual ~TEnColumn() throw() {} + + std::string enData; + std::string nulls; + TTypeId::type type; + int32_t size; + std::string compressorName; + + void __set_enData(const std::string& val) { + enData = val; + } + + void __set_nulls(const std::string& val) { + nulls = val; + } + + void __set_type(const TTypeId::type val) { + type = val; + } + + void __set_size(const int32_t val) { + size = val; + } + + void __set_compressorName(const std::string& val) { + compressorName = val; + } + + bool operator == (const TEnColumn & rhs) const + { + if (!(enData == rhs.enData)) + return false; + if (!(nulls == rhs.nulls)) + return false; + if (!(type == rhs.type)) + return false; + if (!(size == rhs.size)) + return false; + if (!(compressorName == rhs.compressorName)) + return false; + return true; + } + bool operator != (const TEnColumn &rhs) const { + return !(*this == rhs); + } + + bool operator < (const TEnColumn & ) const; + + uint32_t read(::apache::thrift::protocol::TProtocol* iprot); + uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; + +}; + +void swap(TEnColumn &a, TEnColumn &b); + typedef struct _TRowSet__isset { - _TRowSet__isset() : columns(false) {} + _TRowSet__isset() : columns(false), enColumns(false), compressorBitmap(false) {} bool columns; + bool enColumns; + bool compressorBitmap; } _TRowSet__isset; class TRowSet { public: - static const char* ascii_fingerprint; // = "46DA30A870489C7A58105AE0080DAEBF"; - static const uint8_t binary_fingerprint[16]; // = {0x46,0xDA,0x30,0xA8,0x70,0x48,0x9C,0x7A,0x58,0x10,0x5A,0xE0,0x08,0x0D,0xAE,0xBF}; + static const char* ascii_fingerprint; // = "8E980BCEDB71FA696E49ECA856CDDB6F"; + static const uint8_t binary_fingerprint[16]; // = {0x8E,0x98,0x0B,0xCE,0xDB,0x71,0xFA,0x69,0x6E,0x49,0xEC,0xA8,0x56,0xCD,0xDB,0x6F}; - TRowSet() : startRowOffset(0) { + TRowSet() : startRowOffset(0), compressorBitmap() { } virtual ~TRowSet() throw() {} @@ -1661,6 +1729,8 @@ class TRowSet { int64_t startRowOffset; std::vector rows; std::vector columns; + std::vector enColumns; + std::string compressorBitmap; _TRowSet__isset __isset; @@ -1677,6 +1747,16 @@ class TRowSet { __isset.columns = true; } + void __set_enColumns(const std::vector & val) { + enColumns = val; + __isset.enColumns = true; + } + + void __set_compressorBitmap(const std::string& val) { + compressorBitmap = val; + __isset.compressorBitmap = true; + } + bool operator == (const TRowSet & rhs) const { if (!(startRowOffset == rhs.startRowOffset)) @@ -1687,6 +1767,14 @@ class TRowSet { return false; else if (__isset.columns && !(columns == rhs.columns)) return false; + if (__isset.enColumns != rhs.__isset.enColumns) + return false; + else if (__isset.enColumns && !(enColumns == rhs.enColumns)) + return false; + if (__isset.compressorBitmap != rhs.__isset.compressorBitmap) + return false; + else if (__isset.compressorBitmap && !(compressorBitmap == rhs.compressorBitmap)) + return false; return true; } bool operator != (const TRowSet &rhs) const { @@ -3683,8 +3771,8 @@ typedef struct _TFetchResultsResp__isset { class TFetchResultsResp { public: - static const char* ascii_fingerprint; // = "FC43BC2D6F3B76D4DB0F34226A745C8E"; - static const uint8_t binary_fingerprint[16]; // = {0xFC,0x43,0xBC,0x2D,0x6F,0x3B,0x76,0xD4,0xDB,0x0F,0x34,0x22,0x6A,0x74,0x5C,0x8E}; + static const char* ascii_fingerprint; // = "99C26CD71A5438BD7A817ACDDDF8C79B"; + static const uint8_t binary_fingerprint[16]; // = {0x99,0xC2,0x6C,0xD7,0x1A,0x54,0x38,0xBD,0x7A,0x81,0x7A,0xCD,0xDD,0xF8,0xC7,0x9B}; TFetchResultsResp() : hasMoreRows(0) { } diff --git service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TEnColumn.java service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TEnColumn.java new file mode 100644 index 0000000..53d5b5d --- /dev/null +++ service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TEnColumn.java @@ -0,0 +1,819 @@ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.cli.thrift; + +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TEnColumn implements org.apache.thrift.TBase, java.io.Serializable, Cloneable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TEnColumn"); + + private static final org.apache.thrift.protocol.TField EN_DATA_FIELD_DESC = new org.apache.thrift.protocol.TField("enData", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField NULLS_FIELD_DESC = new org.apache.thrift.protocol.TField("nulls", org.apache.thrift.protocol.TType.STRING, (short)2); + private static final org.apache.thrift.protocol.TField TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("type", org.apache.thrift.protocol.TType.I32, (short)3); + private static final org.apache.thrift.protocol.TField SIZE_FIELD_DESC = new org.apache.thrift.protocol.TField("size", org.apache.thrift.protocol.TType.I32, (short)4); + private static final org.apache.thrift.protocol.TField COMPRESSOR_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("compressorName", org.apache.thrift.protocol.TType.STRING, (short)5); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TEnColumnStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TEnColumnTupleSchemeFactory()); + } + + private ByteBuffer enData; // required + private ByteBuffer nulls; // required + private TTypeId type; // required + private int size; // required + private String compressorName; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + EN_DATA((short)1, "enData"), + NULLS((short)2, "nulls"), + /** + * + * @see TTypeId + */ + TYPE((short)3, "type"), + SIZE((short)4, "size"), + COMPRESSOR_NAME((short)5, "compressorName"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // EN_DATA + return EN_DATA; + case 2: // NULLS + return NULLS; + case 3: // TYPE + return TYPE; + case 4: // SIZE + return SIZE; + case 5: // COMPRESSOR_NAME + return COMPRESSOR_NAME; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __SIZE_ISSET_ID = 0; + private byte __isset_bitfield = 0; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.EN_DATA, new org.apache.thrift.meta_data.FieldMetaData("enData", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); + tmpMap.put(_Fields.NULLS, new org.apache.thrift.meta_data.FieldMetaData("nulls", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); + tmpMap.put(_Fields.TYPE, new org.apache.thrift.meta_data.FieldMetaData("type", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TTypeId.class))); + tmpMap.put(_Fields.SIZE, new org.apache.thrift.meta_data.FieldMetaData("size", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.COMPRESSOR_NAME, new org.apache.thrift.meta_data.FieldMetaData("compressorName", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TEnColumn.class, metaDataMap); + } + + public TEnColumn() { + } + + public TEnColumn( + ByteBuffer enData, + ByteBuffer nulls, + TTypeId type, + int size, + String compressorName) + { + this(); + this.enData = enData; + this.nulls = nulls; + this.type = type; + this.size = size; + setSizeIsSet(true); + this.compressorName = compressorName; + } + + /** + * Performs a deep copy on other. + */ + public TEnColumn(TEnColumn other) { + __isset_bitfield = other.__isset_bitfield; + if (other.isSetEnData()) { + this.enData = org.apache.thrift.TBaseHelper.copyBinary(other.enData); +; + } + if (other.isSetNulls()) { + this.nulls = org.apache.thrift.TBaseHelper.copyBinary(other.nulls); +; + } + if (other.isSetType()) { + this.type = other.type; + } + this.size = other.size; + if (other.isSetCompressorName()) { + this.compressorName = other.compressorName; + } + } + + public TEnColumn deepCopy() { + return new TEnColumn(this); + } + + @Override + public void clear() { + this.enData = null; + this.nulls = null; + this.type = null; + setSizeIsSet(false); + this.size = 0; + this.compressorName = null; + } + + public byte[] getEnData() { + setEnData(org.apache.thrift.TBaseHelper.rightSize(enData)); + return enData == null ? null : enData.array(); + } + + public ByteBuffer bufferForEnData() { + return enData; + } + + public void setEnData(byte[] enData) { + setEnData(enData == null ? (ByteBuffer)null : ByteBuffer.wrap(enData)); + } + + public void setEnData(ByteBuffer enData) { + this.enData = enData; + } + + public void unsetEnData() { + this.enData = null; + } + + /** Returns true if field enData is set (has been assigned a value) and false otherwise */ + public boolean isSetEnData() { + return this.enData != null; + } + + public void setEnDataIsSet(boolean value) { + if (!value) { + this.enData = null; + } + } + + public byte[] getNulls() { + setNulls(org.apache.thrift.TBaseHelper.rightSize(nulls)); + return nulls == null ? null : nulls.array(); + } + + public ByteBuffer bufferForNulls() { + return nulls; + } + + public void setNulls(byte[] nulls) { + setNulls(nulls == null ? (ByteBuffer)null : ByteBuffer.wrap(nulls)); + } + + public void setNulls(ByteBuffer nulls) { + this.nulls = nulls; + } + + public void unsetNulls() { + this.nulls = null; + } + + /** Returns true if field nulls is set (has been assigned a value) and false otherwise */ + public boolean isSetNulls() { + return this.nulls != null; + } + + public void setNullsIsSet(boolean value) { + if (!value) { + this.nulls = null; + } + } + + /** + * + * @see TTypeId + */ + public TTypeId getType() { + return this.type; + } + + /** + * + * @see TTypeId + */ + public void setType(TTypeId type) { + this.type = type; + } + + public void unsetType() { + this.type = null; + } + + /** Returns true if field type is set (has been assigned a value) and false otherwise */ + public boolean isSetType() { + return this.type != null; + } + + public void setTypeIsSet(boolean value) { + if (!value) { + this.type = null; + } + } + + public int getSize() { + return this.size; + } + + public void setSize(int size) { + this.size = size; + setSizeIsSet(true); + } + + public void unsetSize() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __SIZE_ISSET_ID); + } + + /** Returns true if field size is set (has been assigned a value) and false otherwise */ + public boolean isSetSize() { + return EncodingUtils.testBit(__isset_bitfield, __SIZE_ISSET_ID); + } + + public void setSizeIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SIZE_ISSET_ID, value); + } + + public String getCompressorName() { + return this.compressorName; + } + + public void setCompressorName(String compressorName) { + this.compressorName = compressorName; + } + + public void unsetCompressorName() { + this.compressorName = null; + } + + /** Returns true if field compressorName is set (has been assigned a value) and false otherwise */ + public boolean isSetCompressorName() { + return this.compressorName != null; + } + + public void setCompressorNameIsSet(boolean value) { + if (!value) { + this.compressorName = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case EN_DATA: + if (value == null) { + unsetEnData(); + } else { + setEnData((ByteBuffer)value); + } + break; + + case NULLS: + if (value == null) { + unsetNulls(); + } else { + setNulls((ByteBuffer)value); + } + break; + + case TYPE: + if (value == null) { + unsetType(); + } else { + setType((TTypeId)value); + } + break; + + case SIZE: + if (value == null) { + unsetSize(); + } else { + setSize((Integer)value); + } + break; + + case COMPRESSOR_NAME: + if (value == null) { + unsetCompressorName(); + } else { + setCompressorName((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case EN_DATA: + return getEnData(); + + case NULLS: + return getNulls(); + + case TYPE: + return getType(); + + case SIZE: + return Integer.valueOf(getSize()); + + case COMPRESSOR_NAME: + return getCompressorName(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case EN_DATA: + return isSetEnData(); + case NULLS: + return isSetNulls(); + case TYPE: + return isSetType(); + case SIZE: + return isSetSize(); + case COMPRESSOR_NAME: + return isSetCompressorName(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TEnColumn) + return this.equals((TEnColumn)that); + return false; + } + + public boolean equals(TEnColumn that) { + if (that == null) + return false; + + boolean this_present_enData = true && this.isSetEnData(); + boolean that_present_enData = true && that.isSetEnData(); + if (this_present_enData || that_present_enData) { + if (!(this_present_enData && that_present_enData)) + return false; + if (!this.enData.equals(that.enData)) + return false; + } + + boolean this_present_nulls = true && this.isSetNulls(); + boolean that_present_nulls = true && that.isSetNulls(); + if (this_present_nulls || that_present_nulls) { + if (!(this_present_nulls && that_present_nulls)) + return false; + if (!this.nulls.equals(that.nulls)) + return false; + } + + boolean this_present_type = true && this.isSetType(); + boolean that_present_type = true && that.isSetType(); + if (this_present_type || that_present_type) { + if (!(this_present_type && that_present_type)) + return false; + if (!this.type.equals(that.type)) + return false; + } + + boolean this_present_size = true; + boolean that_present_size = true; + if (this_present_size || that_present_size) { + if (!(this_present_size && that_present_size)) + return false; + if (this.size != that.size) + return false; + } + + boolean this_present_compressorName = true && this.isSetCompressorName(); + boolean that_present_compressorName = true && that.isSetCompressorName(); + if (this_present_compressorName || that_present_compressorName) { + if (!(this_present_compressorName && that_present_compressorName)) + return false; + if (!this.compressorName.equals(that.compressorName)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + HashCodeBuilder builder = new HashCodeBuilder(); + + boolean present_enData = true && (isSetEnData()); + builder.append(present_enData); + if (present_enData) + builder.append(enData); + + boolean present_nulls = true && (isSetNulls()); + builder.append(present_nulls); + if (present_nulls) + builder.append(nulls); + + boolean present_type = true && (isSetType()); + builder.append(present_type); + if (present_type) + builder.append(type.getValue()); + + boolean present_size = true; + builder.append(present_size); + if (present_size) + builder.append(size); + + boolean present_compressorName = true && (isSetCompressorName()); + builder.append(present_compressorName); + if (present_compressorName) + builder.append(compressorName); + + return builder.toHashCode(); + } + + public int compareTo(TEnColumn other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + TEnColumn typedOther = (TEnColumn)other; + + lastComparison = Boolean.valueOf(isSetEnData()).compareTo(typedOther.isSetEnData()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetEnData()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.enData, typedOther.enData); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetNulls()).compareTo(typedOther.isSetNulls()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetNulls()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nulls, typedOther.nulls); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetType()).compareTo(typedOther.isSetType()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetType()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.type, typedOther.type); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetSize()).compareTo(typedOther.isSetSize()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSize()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.size, typedOther.size); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetCompressorName()).compareTo(typedOther.isSetCompressorName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetCompressorName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.compressorName, typedOther.compressorName); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TEnColumn("); + boolean first = true; + + sb.append("enData:"); + if (this.enData == null) { + sb.append("null"); + } else { + org.apache.thrift.TBaseHelper.toString(this.enData, sb); + } + first = false; + if (!first) sb.append(", "); + sb.append("nulls:"); + if (this.nulls == null) { + sb.append("null"); + } else { + org.apache.thrift.TBaseHelper.toString(this.nulls, sb); + } + first = false; + if (!first) sb.append(", "); + sb.append("type:"); + if (this.type == null) { + sb.append("null"); + } else { + sb.append(this.type); + } + first = false; + if (!first) sb.append(", "); + sb.append("size:"); + sb.append(this.size); + first = false; + if (!first) sb.append(", "); + sb.append("compressorName:"); + if (this.compressorName == null) { + sb.append("null"); + } else { + sb.append(this.compressorName); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetEnData()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'enData' is unset! Struct:" + toString()); + } + + if (!isSetNulls()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'nulls' is unset! Struct:" + toString()); + } + + if (!isSetType()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'type' is unset! Struct:" + toString()); + } + + if (!isSetSize()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'size' is unset! Struct:" + toString()); + } + + if (!isSetCompressorName()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'compressorName' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TEnColumnStandardSchemeFactory implements SchemeFactory { + public TEnColumnStandardScheme getScheme() { + return new TEnColumnStandardScheme(); + } + } + + private static class TEnColumnStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TEnColumn struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // EN_DATA + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.enData = iprot.readBinary(); + struct.setEnDataIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // NULLS + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.nulls = iprot.readBinary(); + struct.setNullsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // TYPE + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.type = TTypeId.findByValue(iprot.readI32()); + struct.setTypeIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // SIZE + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.size = iprot.readI32(); + struct.setSizeIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // COMPRESSOR_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.compressorName = iprot.readString(); + struct.setCompressorNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TEnColumn struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.enData != null) { + oprot.writeFieldBegin(EN_DATA_FIELD_DESC); + oprot.writeBinary(struct.enData); + oprot.writeFieldEnd(); + } + if (struct.nulls != null) { + oprot.writeFieldBegin(NULLS_FIELD_DESC); + oprot.writeBinary(struct.nulls); + oprot.writeFieldEnd(); + } + if (struct.type != null) { + oprot.writeFieldBegin(TYPE_FIELD_DESC); + oprot.writeI32(struct.type.getValue()); + oprot.writeFieldEnd(); + } + oprot.writeFieldBegin(SIZE_FIELD_DESC); + oprot.writeI32(struct.size); + oprot.writeFieldEnd(); + if (struct.compressorName != null) { + oprot.writeFieldBegin(COMPRESSOR_NAME_FIELD_DESC); + oprot.writeString(struct.compressorName); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TEnColumnTupleSchemeFactory implements SchemeFactory { + public TEnColumnTupleScheme getScheme() { + return new TEnColumnTupleScheme(); + } + } + + private static class TEnColumnTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TEnColumn struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeBinary(struct.enData); + oprot.writeBinary(struct.nulls); + oprot.writeI32(struct.type.getValue()); + oprot.writeI32(struct.size); + oprot.writeString(struct.compressorName); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TEnColumn struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.enData = iprot.readBinary(); + struct.setEnDataIsSet(true); + struct.nulls = iprot.readBinary(); + struct.setNullsIsSet(true); + struct.type = TTypeId.findByValue(iprot.readI32()); + struct.setTypeIsSet(true); + struct.size = iprot.readI32(); + struct.setSizeIsSet(true); + struct.compressorName = iprot.readString(); + struct.setCompressorNameIsSet(true); + } + } + +} + diff --git service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TExecuteStatementReq.java service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TExecuteStatementReq.java index 4f157ad..0d4ea94 100644 --- service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TExecuteStatementReq.java +++ service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TExecuteStatementReq.java @@ -624,15 +624,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, TExecuteStatementRe case 3: // CONF_OVERLAY if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map162 = iprot.readMapBegin(); - struct.confOverlay = new HashMap(2*_map162.size); - for (int _i163 = 0; _i163 < _map162.size; ++_i163) + org.apache.thrift.protocol.TMap _map170 = iprot.readMapBegin(); + struct.confOverlay = new HashMap(2*_map170.size); + for (int _i171 = 0; _i171 < _map170.size; ++_i171) { - String _key164; // required - String _val165; // required - _key164 = iprot.readString(); - _val165 = iprot.readString(); - struct.confOverlay.put(_key164, _val165); + String _key172; // required + String _val173; // required + _key172 = iprot.readString(); + _val173 = iprot.readString(); + struct.confOverlay.put(_key172, _val173); } iprot.readMapEnd(); } @@ -677,10 +677,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, TExecuteStatementR oprot.writeFieldBegin(CONF_OVERLAY_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.confOverlay.size())); - for (Map.Entry _iter166 : struct.confOverlay.entrySet()) + for (Map.Entry _iter174 : struct.confOverlay.entrySet()) { - oprot.writeString(_iter166.getKey()); - oprot.writeString(_iter166.getValue()); + oprot.writeString(_iter174.getKey()); + oprot.writeString(_iter174.getValue()); } oprot.writeMapEnd(); } @@ -722,10 +722,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, TExecuteStatementRe if (struct.isSetConfOverlay()) { { oprot.writeI32(struct.confOverlay.size()); - for (Map.Entry _iter167 : struct.confOverlay.entrySet()) + for (Map.Entry _iter175 : struct.confOverlay.entrySet()) { - oprot.writeString(_iter167.getKey()); - oprot.writeString(_iter167.getValue()); + oprot.writeString(_iter175.getKey()); + oprot.writeString(_iter175.getValue()); } } } @@ -745,15 +745,15 @@ public void read(org.apache.thrift.protocol.TProtocol prot, TExecuteStatementReq BitSet incoming = iprot.readBitSet(2); if (incoming.get(0)) { { - org.apache.thrift.protocol.TMap _map168 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.confOverlay = new HashMap(2*_map168.size); - for (int _i169 = 0; _i169 < _map168.size; ++_i169) + org.apache.thrift.protocol.TMap _map176 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.confOverlay = new HashMap(2*_map176.size); + for (int _i177 = 0; _i177 < _map176.size; ++_i177) { - String _key170; // required - String _val171; // required - _key170 = iprot.readString(); - _val171 = iprot.readString(); - struct.confOverlay.put(_key170, _val171); + String _key178; // required + String _val179; // required + _key178 = iprot.readString(); + _val179 = iprot.readString(); + struct.confOverlay.put(_key178, _val179); } } struct.setConfOverlayIsSet(true); diff --git service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetTablesReq.java service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetTablesReq.java index c973fcc..34e22c1 100644 --- service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetTablesReq.java +++ service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TGetTablesReq.java @@ -711,13 +711,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, TGetTablesReq struc case 5: // TABLE_TYPES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list172 = iprot.readListBegin(); - struct.tableTypes = new ArrayList(_list172.size); - for (int _i173 = 0; _i173 < _list172.size; ++_i173) + org.apache.thrift.protocol.TList _list180 = iprot.readListBegin(); + struct.tableTypes = new ArrayList(_list180.size); + for (int _i181 = 0; _i181 < _list180.size; ++_i181) { - String _elem174; // optional - _elem174 = iprot.readString(); - struct.tableTypes.add(_elem174); + String _elem182; // required + _elem182 = iprot.readString(); + struct.tableTypes.add(_elem182); } iprot.readListEnd(); } @@ -770,9 +770,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, TGetTablesReq stru oprot.writeFieldBegin(TABLE_TYPES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tableTypes.size())); - for (String _iter175 : struct.tableTypes) + for (String _iter183 : struct.tableTypes) { - oprot.writeString(_iter175); + oprot.writeString(_iter183); } oprot.writeListEnd(); } @@ -823,9 +823,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, TGetTablesReq struc if (struct.isSetTableTypes()) { { oprot.writeI32(struct.tableTypes.size()); - for (String _iter176 : struct.tableTypes) + for (String _iter184 : struct.tableTypes) { - oprot.writeString(_iter176); + oprot.writeString(_iter184); } } } @@ -852,13 +852,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, TGetTablesReq struct } if (incoming.get(3)) { { - org.apache.thrift.protocol.TList _list177 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.tableTypes = new ArrayList(_list177.size); - for (int _i178 = 0; _i178 < _list177.size; ++_i178) + org.apache.thrift.protocol.TList _list185 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.tableTypes = new ArrayList(_list185.size); + for (int _i186 = 0; _i186 < _list185.size; ++_i186) { - String _elem179; // optional - _elem179 = iprot.readString(); - struct.tableTypes.add(_elem179); + String _elem187; // required + _elem187 = iprot.readString(); + struct.tableTypes.add(_elem187); } } struct.setTableTypesIsSet(true); diff --git service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TOpenSessionReq.java service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TOpenSessionReq.java index c048161..d540607 100644 --- service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TOpenSessionReq.java +++ service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TOpenSessionReq.java @@ -638,15 +638,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, TOpenSessionReq str case 4: // CONFIGURATION if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map142 = iprot.readMapBegin(); - struct.configuration = new HashMap(2*_map142.size); - for (int _i143 = 0; _i143 < _map142.size; ++_i143) + org.apache.thrift.protocol.TMap _map150 = iprot.readMapBegin(); + struct.configuration = new HashMap(2*_map150.size); + for (int _i151 = 0; _i151 < _map150.size; ++_i151) { - String _key144; // required - String _val145; // required - _key144 = iprot.readString(); - _val145 = iprot.readString(); - struct.configuration.put(_key144, _val145); + String _key152; // required + String _val153; // required + _key152 = iprot.readString(); + _val153 = iprot.readString(); + struct.configuration.put(_key152, _val153); } iprot.readMapEnd(); } @@ -692,10 +692,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, TOpenSessionReq st oprot.writeFieldBegin(CONFIGURATION_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.configuration.size())); - for (Map.Entry _iter146 : struct.configuration.entrySet()) + for (Map.Entry _iter154 : struct.configuration.entrySet()) { - oprot.writeString(_iter146.getKey()); - oprot.writeString(_iter146.getValue()); + oprot.writeString(_iter154.getKey()); + oprot.writeString(_iter154.getValue()); } oprot.writeMapEnd(); } @@ -740,10 +740,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, TOpenSessionReq str if (struct.isSetConfiguration()) { { oprot.writeI32(struct.configuration.size()); - for (Map.Entry _iter147 : struct.configuration.entrySet()) + for (Map.Entry _iter155 : struct.configuration.entrySet()) { - oprot.writeString(_iter147.getKey()); - oprot.writeString(_iter147.getValue()); + oprot.writeString(_iter155.getKey()); + oprot.writeString(_iter155.getValue()); } } } @@ -765,15 +765,15 @@ public void read(org.apache.thrift.protocol.TProtocol prot, TOpenSessionReq stru } if (incoming.get(2)) { { - org.apache.thrift.protocol.TMap _map148 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.configuration = new HashMap(2*_map148.size); - for (int _i149 = 0; _i149 < _map148.size; ++_i149) + org.apache.thrift.protocol.TMap _map156 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.configuration = new HashMap(2*_map156.size); + for (int _i157 = 0; _i157 < _map156.size; ++_i157) { - String _key150; // required - String _val151; // required - _key150 = iprot.readString(); - _val151 = iprot.readString(); - struct.configuration.put(_key150, _val151); + String _key158; // required + String _val159; // required + _key158 = iprot.readString(); + _val159 = iprot.readString(); + struct.configuration.put(_key158, _val159); } } struct.setConfigurationIsSet(true); diff --git service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TOpenSessionResp.java service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TOpenSessionResp.java index 351f78b..150f416 100644 --- service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TOpenSessionResp.java +++ service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TOpenSessionResp.java @@ -650,15 +650,15 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, TOpenSessionResp st case 4: // CONFIGURATION if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { { - org.apache.thrift.protocol.TMap _map152 = iprot.readMapBegin(); - struct.configuration = new HashMap(2*_map152.size); - for (int _i153 = 0; _i153 < _map152.size; ++_i153) + org.apache.thrift.protocol.TMap _map160 = iprot.readMapBegin(); + struct.configuration = new HashMap(2*_map160.size); + for (int _i161 = 0; _i161 < _map160.size; ++_i161) { - String _key154; // required - String _val155; // required - _key154 = iprot.readString(); - _val155 = iprot.readString(); - struct.configuration.put(_key154, _val155); + String _key162; // required + String _val163; // required + _key162 = iprot.readString(); + _val163 = iprot.readString(); + struct.configuration.put(_key162, _val163); } iprot.readMapEnd(); } @@ -702,10 +702,10 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, TOpenSessionResp s oprot.writeFieldBegin(CONFIGURATION_FIELD_DESC); { oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.configuration.size())); - for (Map.Entry _iter156 : struct.configuration.entrySet()) + for (Map.Entry _iter164 : struct.configuration.entrySet()) { - oprot.writeString(_iter156.getKey()); - oprot.writeString(_iter156.getValue()); + oprot.writeString(_iter164.getKey()); + oprot.writeString(_iter164.getValue()); } oprot.writeMapEnd(); } @@ -745,10 +745,10 @@ public void write(org.apache.thrift.protocol.TProtocol prot, TOpenSessionResp st if (struct.isSetConfiguration()) { { oprot.writeI32(struct.configuration.size()); - for (Map.Entry _iter157 : struct.configuration.entrySet()) + for (Map.Entry _iter165 : struct.configuration.entrySet()) { - oprot.writeString(_iter157.getKey()); - oprot.writeString(_iter157.getValue()); + oprot.writeString(_iter165.getKey()); + oprot.writeString(_iter165.getValue()); } } } @@ -770,15 +770,15 @@ public void read(org.apache.thrift.protocol.TProtocol prot, TOpenSessionResp str } if (incoming.get(1)) { { - org.apache.thrift.protocol.TMap _map158 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.configuration = new HashMap(2*_map158.size); - for (int _i159 = 0; _i159 < _map158.size; ++_i159) + org.apache.thrift.protocol.TMap _map166 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.configuration = new HashMap(2*_map166.size); + for (int _i167 = 0; _i167 < _map166.size; ++_i167) { - String _key160; // required - String _val161; // required - _key160 = iprot.readString(); - _val161 = iprot.readString(); - struct.configuration.put(_key160, _val161); + String _key168; // required + String _val169; // required + _key168 = iprot.readString(); + _val169 = iprot.readString(); + struct.configuration.put(_key168, _val169); } } struct.setConfigurationIsSet(true); diff --git service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TProtocolVersion.java service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TProtocolVersion.java index a4279d2..f960338 100644 --- service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TProtocolVersion.java +++ service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TProtocolVersion.java @@ -19,7 +19,8 @@ HIVE_CLI_SERVICE_PROTOCOL_V5(4), HIVE_CLI_SERVICE_PROTOCOL_V6(5), HIVE_CLI_SERVICE_PROTOCOL_V7(6), - HIVE_CLI_SERVICE_PROTOCOL_V8(7); + HIVE_CLI_SERVICE_PROTOCOL_V8(7), + HIVE_CLI_SERVICE_PROTOCOL_V9(8); private final int value; @@ -56,6 +57,8 @@ public static TProtocolVersion findByValue(int value) { return HIVE_CLI_SERVICE_PROTOCOL_V7; case 7: return HIVE_CLI_SERVICE_PROTOCOL_V8; + case 8: + return HIVE_CLI_SERVICE_PROTOCOL_V9; default: return null; } diff --git service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TRowSet.java service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TRowSet.java index d16c8a4..9285d8d 100644 --- service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TRowSet.java +++ service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TRowSet.java @@ -37,6 +37,8 @@ private static final org.apache.thrift.protocol.TField START_ROW_OFFSET_FIELD_DESC = new org.apache.thrift.protocol.TField("startRowOffset", org.apache.thrift.protocol.TType.I64, (short)1); private static final org.apache.thrift.protocol.TField ROWS_FIELD_DESC = new org.apache.thrift.protocol.TField("rows", org.apache.thrift.protocol.TType.LIST, (short)2); private static final org.apache.thrift.protocol.TField COLUMNS_FIELD_DESC = new org.apache.thrift.protocol.TField("columns", org.apache.thrift.protocol.TType.LIST, (short)3); + private static final org.apache.thrift.protocol.TField EN_COLUMNS_FIELD_DESC = new org.apache.thrift.protocol.TField("enColumns", org.apache.thrift.protocol.TType.LIST, (short)4); + private static final org.apache.thrift.protocol.TField COMPRESSOR_BITMAP_FIELD_DESC = new org.apache.thrift.protocol.TField("compressorBitmap", org.apache.thrift.protocol.TType.STRING, (short)5); private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); static { @@ -47,12 +49,16 @@ private long startRowOffset; // required private List rows; // required private List columns; // optional + private List enColumns; // optional + private ByteBuffer compressorBitmap; // optional /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ public enum _Fields implements org.apache.thrift.TFieldIdEnum { START_ROW_OFFSET((short)1, "startRowOffset"), ROWS((short)2, "rows"), - COLUMNS((short)3, "columns"); + COLUMNS((short)3, "columns"), + EN_COLUMNS((short)4, "enColumns"), + COMPRESSOR_BITMAP((short)5, "compressorBitmap"); private static final Map byName = new HashMap(); @@ -73,6 +79,10 @@ public static _Fields findByThriftId(int fieldId) { return ROWS; case 3: // COLUMNS return COLUMNS; + case 4: // EN_COLUMNS + return EN_COLUMNS; + case 5: // COMPRESSOR_BITMAP + return COMPRESSOR_BITMAP; default: return null; } @@ -115,7 +125,7 @@ public String getFieldName() { // isset id assignments private static final int __STARTROWOFFSET_ISSET_ID = 0; private byte __isset_bitfield = 0; - private _Fields optionals[] = {_Fields.COLUMNS}; + private _Fields optionals[] = {_Fields.COLUMNS,_Fields.EN_COLUMNS,_Fields.COMPRESSOR_BITMAP}; public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; static { Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); @@ -127,6 +137,11 @@ public String getFieldName() { tmpMap.put(_Fields.COLUMNS, new org.apache.thrift.meta_data.FieldMetaData("columns", org.apache.thrift.TFieldRequirementType.OPTIONAL, new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TColumn.class)))); + tmpMap.put(_Fields.EN_COLUMNS, new org.apache.thrift.meta_data.FieldMetaData("enColumns", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TEnColumn.class)))); + tmpMap.put(_Fields.COMPRESSOR_BITMAP, new org.apache.thrift.meta_data.FieldMetaData("compressorBitmap", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); metaDataMap = Collections.unmodifiableMap(tmpMap); org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TRowSet.class, metaDataMap); } @@ -164,6 +179,17 @@ public TRowSet(TRowSet other) { } this.columns = __this__columns; } + if (other.isSetEnColumns()) { + List __this__enColumns = new ArrayList(); + for (TEnColumn other_element : other.enColumns) { + __this__enColumns.add(new TEnColumn(other_element)); + } + this.enColumns = __this__enColumns; + } + if (other.isSetCompressorBitmap()) { + this.compressorBitmap = org.apache.thrift.TBaseHelper.copyBinary(other.compressorBitmap); +; + } } public TRowSet deepCopy() { @@ -176,6 +202,8 @@ public void clear() { this.startRowOffset = 0; this.rows = null; this.columns = null; + this.enColumns = null; + this.compressorBitmap = null; } public long getStartRowOffset() { @@ -276,6 +304,76 @@ public void setColumnsIsSet(boolean value) { } } + public int getEnColumnsSize() { + return (this.enColumns == null) ? 0 : this.enColumns.size(); + } + + public java.util.Iterator getEnColumnsIterator() { + return (this.enColumns == null) ? null : this.enColumns.iterator(); + } + + public void addToEnColumns(TEnColumn elem) { + if (this.enColumns == null) { + this.enColumns = new ArrayList(); + } + this.enColumns.add(elem); + } + + public List getEnColumns() { + return this.enColumns; + } + + public void setEnColumns(List enColumns) { + this.enColumns = enColumns; + } + + public void unsetEnColumns() { + this.enColumns = null; + } + + /** Returns true if field enColumns is set (has been assigned a value) and false otherwise */ + public boolean isSetEnColumns() { + return this.enColumns != null; + } + + public void setEnColumnsIsSet(boolean value) { + if (!value) { + this.enColumns = null; + } + } + + public byte[] getCompressorBitmap() { + setCompressorBitmap(org.apache.thrift.TBaseHelper.rightSize(compressorBitmap)); + return compressorBitmap == null ? null : compressorBitmap.array(); + } + + public ByteBuffer bufferForCompressorBitmap() { + return compressorBitmap; + } + + public void setCompressorBitmap(byte[] compressorBitmap) { + setCompressorBitmap(compressorBitmap == null ? (ByteBuffer)null : ByteBuffer.wrap(compressorBitmap)); + } + + public void setCompressorBitmap(ByteBuffer compressorBitmap) { + this.compressorBitmap = compressorBitmap; + } + + public void unsetCompressorBitmap() { + this.compressorBitmap = null; + } + + /** Returns true if field compressorBitmap is set (has been assigned a value) and false otherwise */ + public boolean isSetCompressorBitmap() { + return this.compressorBitmap != null; + } + + public void setCompressorBitmapIsSet(boolean value) { + if (!value) { + this.compressorBitmap = null; + } + } + public void setFieldValue(_Fields field, Object value) { switch (field) { case START_ROW_OFFSET: @@ -302,6 +400,22 @@ public void setFieldValue(_Fields field, Object value) { } break; + case EN_COLUMNS: + if (value == null) { + unsetEnColumns(); + } else { + setEnColumns((List)value); + } + break; + + case COMPRESSOR_BITMAP: + if (value == null) { + unsetCompressorBitmap(); + } else { + setCompressorBitmap((ByteBuffer)value); + } + break; + } } @@ -316,6 +430,12 @@ public Object getFieldValue(_Fields field) { case COLUMNS: return getColumns(); + case EN_COLUMNS: + return getEnColumns(); + + case COMPRESSOR_BITMAP: + return getCompressorBitmap(); + } throw new IllegalStateException(); } @@ -333,6 +453,10 @@ public boolean isSet(_Fields field) { return isSetRows(); case COLUMNS: return isSetColumns(); + case EN_COLUMNS: + return isSetEnColumns(); + case COMPRESSOR_BITMAP: + return isSetCompressorBitmap(); } throw new IllegalStateException(); } @@ -377,6 +501,24 @@ public boolean equals(TRowSet that) { return false; } + boolean this_present_enColumns = true && this.isSetEnColumns(); + boolean that_present_enColumns = true && that.isSetEnColumns(); + if (this_present_enColumns || that_present_enColumns) { + if (!(this_present_enColumns && that_present_enColumns)) + return false; + if (!this.enColumns.equals(that.enColumns)) + return false; + } + + boolean this_present_compressorBitmap = true && this.isSetCompressorBitmap(); + boolean that_present_compressorBitmap = true && that.isSetCompressorBitmap(); + if (this_present_compressorBitmap || that_present_compressorBitmap) { + if (!(this_present_compressorBitmap && that_present_compressorBitmap)) + return false; + if (!this.compressorBitmap.equals(that.compressorBitmap)) + return false; + } + return true; } @@ -399,6 +541,16 @@ public int hashCode() { if (present_columns) builder.append(columns); + boolean present_enColumns = true && (isSetEnColumns()); + builder.append(present_enColumns); + if (present_enColumns) + builder.append(enColumns); + + boolean present_compressorBitmap = true && (isSetCompressorBitmap()); + builder.append(present_compressorBitmap); + if (present_compressorBitmap) + builder.append(compressorBitmap); + return builder.toHashCode(); } @@ -440,6 +592,26 @@ public int compareTo(TRowSet other) { return lastComparison; } } + lastComparison = Boolean.valueOf(isSetEnColumns()).compareTo(typedOther.isSetEnColumns()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetEnColumns()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.enColumns, typedOther.enColumns); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetCompressorBitmap()).compareTo(typedOther.isSetCompressorBitmap()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetCompressorBitmap()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.compressorBitmap, typedOther.compressorBitmap); + if (lastComparison != 0) { + return lastComparison; + } + } return 0; } @@ -481,6 +653,26 @@ public String toString() { } first = false; } + if (isSetEnColumns()) { + if (!first) sb.append(", "); + sb.append("enColumns:"); + if (this.enColumns == null) { + sb.append("null"); + } else { + sb.append(this.enColumns); + } + first = false; + } + if (isSetCompressorBitmap()) { + if (!first) sb.append(", "); + sb.append("compressorBitmap:"); + if (this.compressorBitmap == null) { + sb.append("null"); + } else { + org.apache.thrift.TBaseHelper.toString(this.compressorBitmap, sb); + } + first = false; + } sb.append(")"); return sb.toString(); } @@ -549,7 +741,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, TRowSet struct) thr struct.rows = new ArrayList(_list118.size); for (int _i119 = 0; _i119 < _list118.size; ++_i119) { - TRow _elem120; // optional + TRow _elem120; // required _elem120 = new TRow(); _elem120.read(iprot); struct.rows.add(_elem120); @@ -568,7 +760,7 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, TRowSet struct) thr struct.columns = new ArrayList(_list121.size); for (int _i122 = 0; _i122 < _list121.size; ++_i122) { - TColumn _elem123; // optional + TColumn _elem123; // required _elem123 = new TColumn(); _elem123.read(iprot); struct.columns.add(_elem123); @@ -580,6 +772,33 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, TRowSet struct) thr org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } break; + case 4: // EN_COLUMNS + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list124 = iprot.readListBegin(); + struct.enColumns = new ArrayList(_list124.size); + for (int _i125 = 0; _i125 < _list124.size; ++_i125) + { + TEnColumn _elem126; // required + _elem126 = new TEnColumn(); + _elem126.read(iprot); + struct.enColumns.add(_elem126); + } + iprot.readListEnd(); + } + struct.setEnColumnsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // COMPRESSOR_BITMAP + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.compressorBitmap = iprot.readBinary(); + struct.setCompressorBitmapIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; default: org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); } @@ -600,9 +819,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, TRowSet struct) th oprot.writeFieldBegin(ROWS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.rows.size())); - for (TRow _iter124 : struct.rows) + for (TRow _iter127 : struct.rows) { - _iter124.write(oprot); + _iter127.write(oprot); } oprot.writeListEnd(); } @@ -613,15 +832,36 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, TRowSet struct) th oprot.writeFieldBegin(COLUMNS_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.columns.size())); - for (TColumn _iter125 : struct.columns) + for (TColumn _iter128 : struct.columns) + { + _iter128.write(oprot); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + } + if (struct.enColumns != null) { + if (struct.isSetEnColumns()) { + oprot.writeFieldBegin(EN_COLUMNS_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.enColumns.size())); + for (TEnColumn _iter129 : struct.enColumns) { - _iter125.write(oprot); + _iter129.write(oprot); } oprot.writeListEnd(); } oprot.writeFieldEnd(); } } + if (struct.compressorBitmap != null) { + if (struct.isSetCompressorBitmap()) { + oprot.writeFieldBegin(COMPRESSOR_BITMAP_FIELD_DESC); + oprot.writeBinary(struct.compressorBitmap); + oprot.writeFieldEnd(); + } + } oprot.writeFieldStop(); oprot.writeStructEnd(); } @@ -642,25 +882,43 @@ public void write(org.apache.thrift.protocol.TProtocol prot, TRowSet struct) thr oprot.writeI64(struct.startRowOffset); { oprot.writeI32(struct.rows.size()); - for (TRow _iter126 : struct.rows) + for (TRow _iter130 : struct.rows) { - _iter126.write(oprot); + _iter130.write(oprot); } } BitSet optionals = new BitSet(); if (struct.isSetColumns()) { optionals.set(0); } - oprot.writeBitSet(optionals, 1); + if (struct.isSetEnColumns()) { + optionals.set(1); + } + if (struct.isSetCompressorBitmap()) { + optionals.set(2); + } + oprot.writeBitSet(optionals, 3); if (struct.isSetColumns()) { { oprot.writeI32(struct.columns.size()); - for (TColumn _iter127 : struct.columns) + for (TColumn _iter131 : struct.columns) { - _iter127.write(oprot); + _iter131.write(oprot); + } + } + } + if (struct.isSetEnColumns()) { + { + oprot.writeI32(struct.enColumns.size()); + for (TEnColumn _iter132 : struct.enColumns) + { + _iter132.write(oprot); } } } + if (struct.isSetCompressorBitmap()) { + oprot.writeBinary(struct.compressorBitmap); + } } @Override @@ -669,32 +927,50 @@ public void read(org.apache.thrift.protocol.TProtocol prot, TRowSet struct) thro struct.startRowOffset = iprot.readI64(); struct.setStartRowOffsetIsSet(true); { - org.apache.thrift.protocol.TList _list128 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.rows = new ArrayList(_list128.size); - for (int _i129 = 0; _i129 < _list128.size; ++_i129) + org.apache.thrift.protocol.TList _list133 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.rows = new ArrayList(_list133.size); + for (int _i134 = 0; _i134 < _list133.size; ++_i134) { - TRow _elem130; // optional - _elem130 = new TRow(); - _elem130.read(iprot); - struct.rows.add(_elem130); + TRow _elem135; // required + _elem135 = new TRow(); + _elem135.read(iprot); + struct.rows.add(_elem135); } } struct.setRowsIsSet(true); - BitSet incoming = iprot.readBitSet(1); + BitSet incoming = iprot.readBitSet(3); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list131 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); - struct.columns = new ArrayList(_list131.size); - for (int _i132 = 0; _i132 < _list131.size; ++_i132) + org.apache.thrift.protocol.TList _list136 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.columns = new ArrayList(_list136.size); + for (int _i137 = 0; _i137 < _list136.size; ++_i137) { - TColumn _elem133; // optional - _elem133 = new TColumn(); - _elem133.read(iprot); - struct.columns.add(_elem133); + TColumn _elem138; // required + _elem138 = new TColumn(); + _elem138.read(iprot); + struct.columns.add(_elem138); } } struct.setColumnsIsSet(true); } + if (incoming.get(1)) { + { + org.apache.thrift.protocol.TList _list139 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.enColumns = new ArrayList(_list139.size); + for (int _i140 = 0; _i140 < _list139.size; ++_i140) + { + TEnColumn _elem141; // required + _elem141 = new TEnColumn(); + _elem141.read(iprot); + struct.enColumns.add(_elem141); + } + } + struct.setEnColumnsIsSet(true); + } + if (incoming.get(2)) { + struct.compressorBitmap = iprot.readBinary(); + struct.setCompressorBitmapIsSet(true); + } } } diff --git service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TStatus.java service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TStatus.java index 24a746e..b47c636 100644 --- service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TStatus.java +++ service/src/gen/thrift/gen-javabean/org/apache/hive/service/cli/thrift/TStatus.java @@ -694,13 +694,13 @@ public void read(org.apache.thrift.protocol.TProtocol iprot, TStatus struct) thr case 2: // INFO_MESSAGES if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { { - org.apache.thrift.protocol.TList _list134 = iprot.readListBegin(); - struct.infoMessages = new ArrayList(_list134.size); - for (int _i135 = 0; _i135 < _list134.size; ++_i135) + org.apache.thrift.protocol.TList _list142 = iprot.readListBegin(); + struct.infoMessages = new ArrayList(_list142.size); + for (int _i143 = 0; _i143 < _list142.size; ++_i143) { - String _elem136; // optional - _elem136 = iprot.readString(); - struct.infoMessages.add(_elem136); + String _elem144; // required + _elem144 = iprot.readString(); + struct.infoMessages.add(_elem144); } iprot.readListEnd(); } @@ -756,9 +756,9 @@ public void write(org.apache.thrift.protocol.TProtocol oprot, TStatus struct) th oprot.writeFieldBegin(INFO_MESSAGES_FIELD_DESC); { oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.infoMessages.size())); - for (String _iter137 : struct.infoMessages) + for (String _iter145 : struct.infoMessages) { - oprot.writeString(_iter137); + oprot.writeString(_iter145); } oprot.writeListEnd(); } @@ -819,9 +819,9 @@ public void write(org.apache.thrift.protocol.TProtocol prot, TStatus struct) thr if (struct.isSetInfoMessages()) { { oprot.writeI32(struct.infoMessages.size()); - for (String _iter138 : struct.infoMessages) + for (String _iter146 : struct.infoMessages) { - oprot.writeString(_iter138); + oprot.writeString(_iter146); } } } @@ -844,13 +844,13 @@ public void read(org.apache.thrift.protocol.TProtocol prot, TStatus struct) thro BitSet incoming = iprot.readBitSet(4); if (incoming.get(0)) { { - org.apache.thrift.protocol.TList _list139 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); - struct.infoMessages = new ArrayList(_list139.size); - for (int _i140 = 0; _i140 < _list139.size; ++_i140) + org.apache.thrift.protocol.TList _list147 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.infoMessages = new ArrayList(_list147.size); + for (int _i148 = 0; _i148 < _list147.size; ++_i148) { - String _elem141; // optional - _elem141 = iprot.readString(); - struct.infoMessages.add(_elem141); + String _elem149; // required + _elem149 = iprot.readString(); + struct.infoMessages.add(_elem149); } } struct.setInfoMessagesIsSet(true); diff --git service/src/gen/thrift/gen-py/TCLIService/ttypes.py service/src/gen/thrift/gen-py/TCLIService/ttypes.py index 068727c..8057910 100644 --- service/src/gen/thrift/gen-py/TCLIService/ttypes.py +++ service/src/gen/thrift/gen-py/TCLIService/ttypes.py @@ -25,6 +25,7 @@ class TProtocolVersion: HIVE_CLI_SERVICE_PROTOCOL_V6 = 5 HIVE_CLI_SERVICE_PROTOCOL_V7 = 6 HIVE_CLI_SERVICE_PROTOCOL_V8 = 7 + HIVE_CLI_SERVICE_PROTOCOL_V9 = 8 _VALUES_TO_NAMES = { 0: "HIVE_CLI_SERVICE_PROTOCOL_V1", @@ -35,6 +36,7 @@ class TProtocolVersion: 5: "HIVE_CLI_SERVICE_PROTOCOL_V6", 6: "HIVE_CLI_SERVICE_PROTOCOL_V7", 7: "HIVE_CLI_SERVICE_PROTOCOL_V8", + 8: "HIVE_CLI_SERVICE_PROTOCOL_V9", } _NAMES_TO_VALUES = { @@ -46,6 +48,7 @@ class TProtocolVersion: "HIVE_CLI_SERVICE_PROTOCOL_V6": 5, "HIVE_CLI_SERVICE_PROTOCOL_V7": 6, "HIVE_CLI_SERVICE_PROTOCOL_V8": 7, + "HIVE_CLI_SERVICE_PROTOCOL_V9": 8, } class TTypeId: @@ -2777,12 +2780,132 @@ def __eq__(self, other): def __ne__(self, other): return not (self == other) +class TEnColumn: + """ + Attributes: + - enData + - nulls + - type + - size + - compressorName + """ + + thrift_spec = ( + None, # 0 + (1, TType.STRING, 'enData', None, None, ), # 1 + (2, TType.STRING, 'nulls', None, None, ), # 2 + (3, TType.I32, 'type', None, None, ), # 3 + (4, TType.I32, 'size', None, None, ), # 4 + (5, TType.STRING, 'compressorName', None, None, ), # 5 + ) + + def __init__(self, enData=None, nulls=None, type=None, size=None, compressorName=None,): + self.enData = enData + self.nulls = nulls + self.type = type + self.size = size + self.compressorName = compressorName + + def read(self, iprot): + if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: + fastbinary.decode_binary(self, iprot.trans, (self.__class__, self.thrift_spec)) + return + iprot.readStructBegin() + while True: + (fname, ftype, fid) = iprot.readFieldBegin() + if ftype == TType.STOP: + break + if fid == 1: + if ftype == TType.STRING: + self.enData = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 2: + if ftype == TType.STRING: + self.nulls = iprot.readString(); + else: + iprot.skip(ftype) + elif fid == 3: + if ftype == TType.I32: + self.type = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 4: + if ftype == TType.I32: + self.size = iprot.readI32(); + else: + iprot.skip(ftype) + elif fid == 5: + if ftype == TType.STRING: + self.compressorName = iprot.readString(); + else: + iprot.skip(ftype) + else: + iprot.skip(ftype) + iprot.readFieldEnd() + iprot.readStructEnd() + + def write(self, oprot): + if oprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and self.thrift_spec is not None and fastbinary is not None: + oprot.trans.write(fastbinary.encode_binary(self, (self.__class__, self.thrift_spec))) + return + oprot.writeStructBegin('TEnColumn') + if self.enData is not None: + oprot.writeFieldBegin('enData', TType.STRING, 1) + oprot.writeString(self.enData) + oprot.writeFieldEnd() + if self.nulls is not None: + oprot.writeFieldBegin('nulls', TType.STRING, 2) + oprot.writeString(self.nulls) + oprot.writeFieldEnd() + if self.type is not None: + oprot.writeFieldBegin('type', TType.I32, 3) + oprot.writeI32(self.type) + oprot.writeFieldEnd() + if self.size is not None: + oprot.writeFieldBegin('size', TType.I32, 4) + oprot.writeI32(self.size) + oprot.writeFieldEnd() + if self.compressorName is not None: + oprot.writeFieldBegin('compressorName', TType.STRING, 5) + oprot.writeString(self.compressorName) + oprot.writeFieldEnd() + oprot.writeFieldStop() + oprot.writeStructEnd() + + def validate(self): + if self.enData is None: + raise TProtocol.TProtocolException(message='Required field enData is unset!') + if self.nulls is None: + raise TProtocol.TProtocolException(message='Required field nulls is unset!') + if self.type is None: + raise TProtocol.TProtocolException(message='Required field type is unset!') + if self.size is None: + raise TProtocol.TProtocolException(message='Required field size is unset!') + if self.compressorName is None: + raise TProtocol.TProtocolException(message='Required field compressorName is unset!') + return + + + def __repr__(self): + L = ['%s=%r' % (key, value) + for key, value in self.__dict__.iteritems()] + return '%s(%s)' % (self.__class__.__name__, ', '.join(L)) + + def __eq__(self, other): + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ + + def __ne__(self, other): + return not (self == other) + class TRowSet: """ Attributes: - startRowOffset - rows - columns + - enColumns + - compressorBitmap """ thrift_spec = ( @@ -2790,12 +2913,16 @@ class TRowSet: (1, TType.I64, 'startRowOffset', None, None, ), # 1 (2, TType.LIST, 'rows', (TType.STRUCT,(TRow, TRow.thrift_spec)), None, ), # 2 (3, TType.LIST, 'columns', (TType.STRUCT,(TColumn, TColumn.thrift_spec)), None, ), # 3 + (4, TType.LIST, 'enColumns', (TType.STRUCT,(TEnColumn, TEnColumn.thrift_spec)), None, ), # 4 + (5, TType.STRING, 'compressorBitmap', None, None, ), # 5 ) - def __init__(self, startRowOffset=None, rows=None, columns=None,): + def __init__(self, startRowOffset=None, rows=None, columns=None, enColumns=None, compressorBitmap=None,): self.startRowOffset = startRowOffset self.rows = rows self.columns = columns + self.enColumns = enColumns + self.compressorBitmap = compressorBitmap def read(self, iprot): if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None: @@ -2833,6 +2960,22 @@ def read(self, iprot): iprot.readListEnd() else: iprot.skip(ftype) + elif fid == 4: + if ftype == TType.LIST: + self.enColumns = [] + (_etype119, _size116) = iprot.readListBegin() + for _i120 in xrange(_size116): + _elem121 = TEnColumn() + _elem121.read(iprot) + self.enColumns.append(_elem121) + iprot.readListEnd() + else: + iprot.skip(ftype) + elif fid == 5: + if ftype == TType.STRING: + self.compressorBitmap = iprot.readString(); + else: + iprot.skip(ftype) else: iprot.skip(ftype) iprot.readFieldEnd() @@ -2850,17 +2993,28 @@ def write(self, oprot): if self.rows is not None: oprot.writeFieldBegin('rows', TType.LIST, 2) oprot.writeListBegin(TType.STRUCT, len(self.rows)) - for iter116 in self.rows: - iter116.write(oprot) + for iter122 in self.rows: + iter122.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() if self.columns is not None: oprot.writeFieldBegin('columns', TType.LIST, 3) oprot.writeListBegin(TType.STRUCT, len(self.columns)) - for iter117 in self.columns: - iter117.write(oprot) + for iter123 in self.columns: + iter123.write(oprot) + oprot.writeListEnd() + oprot.writeFieldEnd() + if self.enColumns is not None: + oprot.writeFieldBegin('enColumns', TType.LIST, 4) + oprot.writeListBegin(TType.STRUCT, len(self.enColumns)) + for iter124 in self.enColumns: + iter124.write(oprot) oprot.writeListEnd() oprot.writeFieldEnd() + if self.compressorBitmap is not None: + oprot.writeFieldBegin('compressorBitmap', TType.STRING, 5) + oprot.writeString(self.compressorBitmap) + oprot.writeFieldEnd() oprot.writeFieldStop() oprot.writeStructEnd() @@ -2926,10 +3080,10 @@ def read(self, iprot): elif fid == 2: if ftype == TType.LIST: self.infoMessages = [] - (_etype121, _size118) = iprot.readListBegin() - for _i122 in xrange(_size118): - _elem123 = iprot.readString(); - self.infoMessages.append(_elem123) + (_etype128, _size125) = iprot.readListBegin() + for _i129 in xrange(_size125): + _elem130 = iprot.readString(); + self.infoMessages.append(_elem130) iprot.readListEnd() else: iprot.skip(ftype) @@ -2965,8 +3119,8 @@ def write(self, oprot): if self.infoMessages is not None: oprot.writeFieldBegin('infoMessages', TType.LIST, 2) oprot.writeListBegin(TType.STRING, len(self.infoMessages)) - for iter124 in self.infoMessages: - oprot.writeString(iter124) + for iter131 in self.infoMessages: + oprot.writeString(iter131) oprot.writeListEnd() oprot.writeFieldEnd() if self.sqlState is not None: @@ -3293,11 +3447,11 @@ def read(self, iprot): elif fid == 4: if ftype == TType.MAP: self.configuration = {} - (_ktype126, _vtype127, _size125 ) = iprot.readMapBegin() - for _i129 in xrange(_size125): - _key130 = iprot.readString(); - _val131 = iprot.readString(); - self.configuration[_key130] = _val131 + (_ktype133, _vtype134, _size132 ) = iprot.readMapBegin() + for _i136 in xrange(_size132): + _key137 = iprot.readString(); + _val138 = iprot.readString(); + self.configuration[_key137] = _val138 iprot.readMapEnd() else: iprot.skip(ftype) @@ -3326,9 +3480,9 @@ def write(self, oprot): if self.configuration is not None: oprot.writeFieldBegin('configuration', TType.MAP, 4) oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.configuration)) - for kiter132,viter133 in self.configuration.items(): - oprot.writeString(kiter132) - oprot.writeString(viter133) + for kiter139,viter140 in self.configuration.items(): + oprot.writeString(kiter139) + oprot.writeString(viter140) oprot.writeMapEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -3403,11 +3557,11 @@ def read(self, iprot): elif fid == 4: if ftype == TType.MAP: self.configuration = {} - (_ktype135, _vtype136, _size134 ) = iprot.readMapBegin() - for _i138 in xrange(_size134): - _key139 = iprot.readString(); - _val140 = iprot.readString(); - self.configuration[_key139] = _val140 + (_ktype142, _vtype143, _size141 ) = iprot.readMapBegin() + for _i145 in xrange(_size141): + _key146 = iprot.readString(); + _val147 = iprot.readString(); + self.configuration[_key146] = _val147 iprot.readMapEnd() else: iprot.skip(ftype) @@ -3436,9 +3590,9 @@ def write(self, oprot): if self.configuration is not None: oprot.writeFieldBegin('configuration', TType.MAP, 4) oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.configuration)) - for kiter141,viter142 in self.configuration.items(): - oprot.writeString(kiter141) - oprot.writeString(viter142) + for kiter148,viter149 in self.configuration.items(): + oprot.writeString(kiter148) + oprot.writeString(viter149) oprot.writeMapEnd() oprot.writeFieldEnd() oprot.writeFieldStop() @@ -3910,11 +4064,11 @@ def read(self, iprot): elif fid == 3: if ftype == TType.MAP: self.confOverlay = {} - (_ktype144, _vtype145, _size143 ) = iprot.readMapBegin() - for _i147 in xrange(_size143): - _key148 = iprot.readString(); - _val149 = iprot.readString(); - self.confOverlay[_key148] = _val149 + (_ktype151, _vtype152, _size150 ) = iprot.readMapBegin() + for _i154 in xrange(_size150): + _key155 = iprot.readString(); + _val156 = iprot.readString(); + self.confOverlay[_key155] = _val156 iprot.readMapEnd() else: iprot.skip(ftype) @@ -3944,9 +4098,9 @@ def write(self, oprot): if self.confOverlay is not None: oprot.writeFieldBegin('confOverlay', TType.MAP, 3) oprot.writeMapBegin(TType.STRING, TType.STRING, len(self.confOverlay)) - for kiter150,viter151 in self.confOverlay.items(): - oprot.writeString(kiter150) - oprot.writeString(viter151) + for kiter157,viter158 in self.confOverlay.items(): + oprot.writeString(kiter157) + oprot.writeString(viter158) oprot.writeMapEnd() oprot.writeFieldEnd() if self.runAsync is not None: @@ -4551,10 +4705,10 @@ def read(self, iprot): elif fid == 5: if ftype == TType.LIST: self.tableTypes = [] - (_etype155, _size152) = iprot.readListBegin() - for _i156 in xrange(_size152): - _elem157 = iprot.readString(); - self.tableTypes.append(_elem157) + (_etype162, _size159) = iprot.readListBegin() + for _i163 in xrange(_size159): + _elem164 = iprot.readString(); + self.tableTypes.append(_elem164) iprot.readListEnd() else: iprot.skip(ftype) @@ -4587,8 +4741,8 @@ def write(self, oprot): if self.tableTypes is not None: oprot.writeFieldBegin('tableTypes', TType.LIST, 5) oprot.writeListBegin(TType.STRING, len(self.tableTypes)) - for iter158 in self.tableTypes: - oprot.writeString(iter158) + for iter165 in self.tableTypes: + oprot.writeString(iter165) oprot.writeListEnd() oprot.writeFieldEnd() oprot.writeFieldStop() diff --git service/src/gen/thrift/gen-rb/t_c_l_i_service_types.rb service/src/gen/thrift/gen-rb/t_c_l_i_service_types.rb index b482533..90edce3 100644 --- service/src/gen/thrift/gen-rb/t_c_l_i_service_types.rb +++ service/src/gen/thrift/gen-rb/t_c_l_i_service_types.rb @@ -15,8 +15,9 @@ module TProtocolVersion HIVE_CLI_SERVICE_PROTOCOL_V6 = 5 HIVE_CLI_SERVICE_PROTOCOL_V7 = 6 HIVE_CLI_SERVICE_PROTOCOL_V8 = 7 - VALUE_MAP = {0 => "HIVE_CLI_SERVICE_PROTOCOL_V1", 1 => "HIVE_CLI_SERVICE_PROTOCOL_V2", 2 => "HIVE_CLI_SERVICE_PROTOCOL_V3", 3 => "HIVE_CLI_SERVICE_PROTOCOL_V4", 4 => "HIVE_CLI_SERVICE_PROTOCOL_V5", 5 => "HIVE_CLI_SERVICE_PROTOCOL_V6", 6 => "HIVE_CLI_SERVICE_PROTOCOL_V7", 7 => "HIVE_CLI_SERVICE_PROTOCOL_V8"} - VALID_VALUES = Set.new([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_V6, HIVE_CLI_SERVICE_PROTOCOL_V7, HIVE_CLI_SERVICE_PROTOCOL_V8]).freeze + HIVE_CLI_SERVICE_PROTOCOL_V9 = 8 + VALUE_MAP = {0 => "HIVE_CLI_SERVICE_PROTOCOL_V1", 1 => "HIVE_CLI_SERVICE_PROTOCOL_V2", 2 => "HIVE_CLI_SERVICE_PROTOCOL_V3", 3 => "HIVE_CLI_SERVICE_PROTOCOL_V4", 4 => "HIVE_CLI_SERVICE_PROTOCOL_V5", 5 => "HIVE_CLI_SERVICE_PROTOCOL_V6", 6 => "HIVE_CLI_SERVICE_PROTOCOL_V7", 7 => "HIVE_CLI_SERVICE_PROTOCOL_V8", 8 => "HIVE_CLI_SERVICE_PROTOCOL_V9"} + VALID_VALUES = Set.new([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_V6, HIVE_CLI_SERVICE_PROTOCOL_V7, HIVE_CLI_SERVICE_PROTOCOL_V8, HIVE_CLI_SERVICE_PROTOCOL_V9]).freeze end module TTypeId @@ -827,16 +828,52 @@ class TColumn < ::Thrift::Union ::Thrift::Union.generate_accessors self end +class TEnColumn + include ::Thrift::Struct, ::Thrift::Struct_Union + ENDATA = 1 + NULLS = 2 + TYPE = 3 + SIZE = 4 + COMPRESSORNAME = 5 + + FIELDS = { + ENDATA => {:type => ::Thrift::Types::STRING, :name => 'enData', :binary => true}, + NULLS => {:type => ::Thrift::Types::STRING, :name => 'nulls', :binary => true}, + TYPE => {:type => ::Thrift::Types::I32, :name => 'type', :enum_class => ::TTypeId}, + SIZE => {:type => ::Thrift::Types::I32, :name => 'size'}, + COMPRESSORNAME => {:type => ::Thrift::Types::STRING, :name => 'compressorName'} + } + + def struct_fields; FIELDS; end + + def validate + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field enData is unset!') unless @enData + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field nulls is unset!') unless @nulls + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field type is unset!') unless @type + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field size is unset!') unless @size + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Required field compressorName is unset!') unless @compressorName + unless @type.nil? || ::TTypeId::VALID_VALUES.include?(@type) + raise ::Thrift::ProtocolException.new(::Thrift::ProtocolException::UNKNOWN, 'Invalid value of field type!') + end + end + + ::Thrift::Struct.generate_accessors self +end + class TRowSet include ::Thrift::Struct, ::Thrift::Struct_Union STARTROWOFFSET = 1 ROWS = 2 COLUMNS = 3 + ENCOLUMNS = 4 + COMPRESSORBITMAP = 5 FIELDS = { STARTROWOFFSET => {:type => ::Thrift::Types::I64, :name => 'startRowOffset'}, ROWS => {:type => ::Thrift::Types::LIST, :name => 'rows', :element => {:type => ::Thrift::Types::STRUCT, :class => ::TRow}}, - COLUMNS => {:type => ::Thrift::Types::LIST, :name => 'columns', :element => {:type => ::Thrift::Types::STRUCT, :class => ::TColumn}, :optional => true} + COLUMNS => {:type => ::Thrift::Types::LIST, :name => 'columns', :element => {:type => ::Thrift::Types::STRUCT, :class => ::TColumn}, :optional => true}, + ENCOLUMNS => {:type => ::Thrift::Types::LIST, :name => 'enColumns', :element => {:type => ::Thrift::Types::STRUCT, :class => ::TEnColumn}, :optional => true}, + COMPRESSORBITMAP => {:type => ::Thrift::Types::STRING, :name => 'compressorBitmap', :binary => true, :optional => true} } def struct_fields; FIELDS; end diff --git service/src/java/org/apache/hive/service/cli/Column.java service/src/java/org/apache/hive/service/cli/Column.java index 2e21f18..a27e7e3 100644 --- service/src/java/org/apache/hive/service/cli/Column.java +++ service/src/java/org/apache/hive/service/cli/Column.java @@ -62,6 +62,42 @@ private List stringVars; private List binaryVars; + public boolean[] getBools() { + return boolVars; + } + + public byte[] getBytes() { + return byteVars; + } + + public short[] getShorts() { + return shortVars; + } + + public int[] getInts() { + return intVars; + } + + public long[] getLongs() { + return longVars; + } + + public double[] getDoubles() { + return doubleVars; + } + + public List getStrings() { + return stringVars; + } + + public List getBinaries() { + return binaryVars; + } + + public ByteBuffer getNulls() { + return ByteBuffer.wrap(toBinary(nulls)); + } + public Column(Type type, BitSet nulls, Object values) { this.type = type; this.nulls = nulls; diff --git service/src/java/org/apache/hive/service/cli/ColumnBasedSet.java service/src/java/org/apache/hive/service/cli/ColumnBasedSet.java index 47a582e..15edbd0 100644 --- service/src/java/org/apache/hive/service/cli/ColumnBasedSet.java +++ service/src/java/org/apache/hive/service/cli/ColumnBasedSet.java @@ -31,10 +31,10 @@ */ public class ColumnBasedSet implements RowSet { - private long startOffset; + protected long startOffset; - private final Type[] types; // non-null only for writing (server-side) - private final List columns; + protected final Type[] types; // non-null only for writing (server-side) + protected final List columns; public ColumnBasedSet(TableSchema schema) { types = schema.toTypes(); @@ -53,7 +53,7 @@ public ColumnBasedSet(TRowSet tRowSet) { startOffset = tRowSet.getStartRowOffset(); } - private ColumnBasedSet(Type[] types, List columns, long startOffset) { + protected ColumnBasedSet(Type[] types, List columns, long startOffset) { this.types = types; this.columns = columns; this.startOffset = startOffset; diff --git service/src/java/org/apache/hive/service/cli/ColumnCompressor.java service/src/java/org/apache/hive/service/cli/ColumnCompressor.java new file mode 100644 index 0000000..2c5e21b --- /dev/null +++ service/src/java/org/apache/hive/service/cli/ColumnCompressor.java @@ -0,0 +1,20 @@ +package org.apache.hive.service.cli; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + + +@InterfaceAudience.Private +@InterfaceStability.Unstable +/* + * An interface that can be implemented to implement a ResultSet Compressor + * isCompressible(Column col) is used to check if a column, of any given type and size, can be compressed or not by an external compressor + * compress(Column col) would accept a column as argument and return a byte array + */ +public interface ColumnCompressor { + public boolean isCompressible(Column col); + public byte[] compress(Column col); + public String getVendor(); + public String getCompressorSet(); +} + diff --git service/src/java/org/apache/hive/service/cli/ColumnCompressorService.java service/src/java/org/apache/hive/service/cli/ColumnCompressorService.java new file mode 100644 index 0000000..bb22f24 --- /dev/null +++ service/src/java/org/apache/hive/service/cli/ColumnCompressorService.java @@ -0,0 +1,56 @@ +package org.apache.hive.service.cli; + +import java.util.Hashtable; +import java.util.Iterator; +import java.util.ServiceLoader; + +public class ColumnCompressorService { + + private static ColumnCompressorService service; + private ServiceLoader loader; + private Hashtable> compressorTable = new Hashtable<>(); + + private ColumnCompressorService() { + loader = ServiceLoader.load(ColumnCompressor.class); + } + + /** + * Get the singleton instance of ColumnCompressorService. + * + * @return A singleton instance of ColumnCompressorService. + */ + public static synchronized ColumnCompressorService getInstance() { + if (service == null) { + service = new ColumnCompressorService(); + Iterator compressors = service.loader.iterator(); + while (compressors.hasNext()) { + ColumnCompressor compressor = compressors.next(); + service.compressorTable.put(compressor.getVendor() + "." + compressor.getCompressorSet(), + compressor.getClass()); + } + } + return service; + } + + /** + * Get a compressor object with the specified vendor and compressorSet name, if the compressor + * class was loaded from CLASSPATH. + * + * @param vendor + * The vendor string of the compressor implementation, provided by the compressor class + * @param compressorSet + * The compressor set string of the compressor implementation, provided by the compressor + * class + * @return A ColumnCompressor implementation object + */ + public ColumnCompressor getCompressor(String vendor, String compressorSet) { + Class compressorClass = compressorTable.get(vendor + "." + + compressorSet); + ColumnCompressor compressor = null; + try { + compressor = compressorClass.newInstance(); + } catch (Exception e) { + } + return compressor; + } +} diff --git service/src/java/org/apache/hive/service/cli/EncodedColumnBasedSet.java service/src/java/org/apache/hive/service/cli/EncodedColumnBasedSet.java new file mode 100644 index 0000000..37454f1 --- /dev/null +++ service/src/java/org/apache/hive/service/cli/EncodedColumnBasedSet.java @@ -0,0 +1,200 @@ +package org.apache.hive.service.cli; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hive.service.cli.thrift.TColumn; +import org.apache.hive.service.cli.thrift.TEnColumn; +import org.apache.hive.service.cli.thrift.TRowSet; +import org.apache.hive.service.cli.thrift.TRow; +import org.json.JSONException; +import org.json.JSONObject; + +import java.util.BitSet; +import java.nio.ByteBuffer; + +public class EncodedColumnBasedSet extends ColumnBasedSet { + /** + * Class that can handle resultSets which are a mix of compressed and uncompressed columns To + * compress columns, it would need a JSON string a client would send(ThriftCLIService.java) The + * JSON string would have vendor, compressorSet and the entryClass (for a plugin) Using them, + * server would know which entry class to call. A All of these entry classes would have to + * implement the interface (ColumnCompressor.java) The compressorInfo is the JSON string in the + * form {"INT_TYPE": {"vendor": (name), "compressorSet":(name), "entryClass": (name)}, ...} If the + * client mentioned the wrong vendor or classname or compressorSet, or it does so correctly but + * the server does not have it, the column would not be compressed The client can find out which + * columns are/aren't compressed by looking at the compressorBitmap The column number that's + * compressed would have it's respective bit set and vice-versa + */ + + private HiveConf hiveConf; + + /* + * Compressors that shouldn't be used specified as csv under + * "hive.resultset.compression.disabled.compressors". + */ + private HashSet disabledCompressors = new HashSet(); + + public EncodedColumnBasedSet(TableSchema schema) { + super(schema); + } + + public EncodedColumnBasedSet(TRowSet tRowSet) { + super(tRowSet); + } + + public EncodedColumnBasedSet(Type[] types, List subset, long startOffset) { + super(types, subset, startOffset); + } + + /** + * Given an index, update the TRowSet with the column (uncompressed) and update the bitmap with + * that index set to false. + * + * @param tRowSet + * a given TRowSet + * @param i + * index for the column which needs to be inserted into TColumns + * @param bitmap + * bitmap that needs to be updated with info that column "i" is not compressed + * + */ + + private void addUncompressedColumn(TRowSet tRowSet, int i, BitSet bitmap) { + tRowSet.addToColumns(columns.get(i).toTColumn()); + bitmap.set(i, false); + } + + /** + * Main function that converts the columns in the RowSet if the compressorInfo points to a valid + * class and the compressor is not part of the disable compressorList (referred to above). + * + */ + @Override + public TRowSet toTRowSet() { + + if (hiveConf == null) { + throw new IllegalStateException("Hive configuration from session not set"); + } + TRowSet tRowSet = new TRowSet(getStartOffset(), new ArrayList()); + tRowSet.setColumns(new ArrayList()); + // A bitmap showing whether each column is compressed or not + BitSet compressorBitmap = new BitSet(); + // Get the JSON string specifying client compressor preferences + String compressorInfo = hiveConf.get("CompressorInfo"); + if (!hiveConf.getBoolVar(ConfVars.HIVE_RESULTSET_COMPRESSION_ENABLED) || compressorInfo == null) { + for (int i = 0; i < columns.size(); i++) { + addUncompressedColumn(tRowSet, i, compressorBitmap); + } + } else { + JSONObject compressorInfoJSON = null; + try { + compressorInfoJSON = new JSONObject(compressorInfo); + } catch (JSONException e) { + } + if (compressorInfoJSON != null) { + for (int i = 0; i < columns.size(); i++) { + // Add this column, possibly compressed, to the row set. + addColumn(tRowSet, i, compressorInfoJSON, compressorBitmap); + } + } else { + for (int i = 0; i < columns.size(); i++) { + addUncompressedColumn(tRowSet, i, compressorBitmap); + } + } + } + ByteBuffer bitmap = ByteBuffer.wrap(compressorBitmap.toByteArray()); + tRowSet.setCompressorBitmap(bitmap); + return tRowSet; + } + + /** + * Add the column at specified index to the row set; compress it before adding if the JSON + * configuration is valid. + * + * @param tRowSet + * Row set to add the column to + * @param i + * Index of the column of add to the row set + * @param compressorInfoJson + * A JSON object containing type-specific compressor configurations + * @param compressorBitmap + * The bitmap to be set to show whether the column is compressed or not + */ + private void addColumn(TRowSet tRowSet, int i, JSONObject compressorInfoJson, + BitSet compressorBitmap) { + String vendor = ""; + String compressorSet = ""; + try { + JSONObject jsonObject = compressorInfoJson.getJSONObject(columns.get(i).getType().toString()); + vendor = jsonObject.getString("vendor"); + compressorSet = jsonObject.getString("compressorSet"); + } catch (JSONException e) { + addUncompressedColumn(tRowSet, i, compressorBitmap); + return; + } + String compressorKey = vendor + "." + compressorSet; + if (disabledCompressors.contains(compressorKey) == true) { + addUncompressedColumn(tRowSet, i, compressorBitmap); + return; + } + ColumnCompressor columnCompressorImpl = null; + // Dynamically load the compressor class specified by the JSON configuration, if + // the compressor jar is present in CLASSPATH. + columnCompressorImpl = ColumnCompressorService.getInstance().getCompressor(vendor, + compressorSet); + if (columnCompressorImpl == null) { + addUncompressedColumn(tRowSet, i, compressorBitmap); + return; + } + // The compressor may not be able to compress the column because the type is not supported, etc. + if (!columnCompressorImpl.isCompressible(columns.get(i))) { + addUncompressedColumn(tRowSet, i, compressorBitmap); + return; + } + // Create the TEnColumn that will be carrying the compressed column data + TEnColumn tEnColumn = new TEnColumn(); + int colSize = columns.get(i).size(); + tEnColumn.setSize(colSize); + tEnColumn.setType(columns.get(i).getType().toTType()); + tEnColumn.setNulls(columns.get(i).getNulls()); + tEnColumn.setCompressorName(compressorSet); + compressorBitmap.set(i, true); + if (colSize == 0) { + tEnColumn.setEnData(new byte[0]); + } else { + tEnColumn.setEnData(columnCompressorImpl.compress(columns.get(i))); + } + tRowSet.addToEnColumns(tEnColumn); + } + + @Override + public EncodedColumnBasedSet extractSubset(int maxRows) { + int numRows = Math.min(numRows(), maxRows); + + List subset = new ArrayList(); + for (int i = 0; i < columns.size(); i++) { + subset.add(columns.get(i).extractSubset(0, numRows)); + } + EncodedColumnBasedSet result = new EncodedColumnBasedSet(types, subset, startOffset); + startOffset += numRows; + return result; + } + + /** + * Pass the Hive session configuration containing client compressor configurations to this object. + * + * @param conf + * Current Hive session configuration to set + */ + public void setConf(HiveConf conf) { + this.hiveConf = conf; + String[] disabledCompressors = this.hiveConf.getVar( + ConfVars.HIVE_RESULTSET_COMPRESSION_DISABLED_COMPRESSORS).split(","); + this.disabledCompressors.addAll(Arrays.asList(disabledCompressors)); + } +} diff --git service/src/java/org/apache/hive/service/cli/RowSetFactory.java service/src/java/org/apache/hive/service/cli/RowSetFactory.java index e8f68ea..40c71d7 100644 --- service/src/java/org/apache/hive/service/cli/RowSetFactory.java +++ service/src/java/org/apache/hive/service/cli/RowSetFactory.java @@ -22,10 +22,14 @@ import org.apache.hive.service.cli.thrift.TRowSet; import static org.apache.hive.service.cli.thrift.TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V6; +import static org.apache.hive.service.cli.thrift.TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V9; public class RowSetFactory { public static RowSet create(TableSchema schema, TProtocolVersion version) { + if (version.getValue() == HIVE_CLI_SERVICE_PROTOCOL_V9.getValue()) { + return new EncodedColumnBasedSet(schema); + } if (version.getValue() >= HIVE_CLI_SERVICE_PROTOCOL_V6.getValue()) { return new ColumnBasedSet(schema); } @@ -33,6 +37,9 @@ public static RowSet create(TableSchema schema, TProtocolVersion version) { } public static RowSet create(TRowSet results, TProtocolVersion version) { + if (version.getValue() == HIVE_CLI_SERVICE_PROTOCOL_V9.getValue()) { + return new EncodedColumnBasedSet(results); + } if (version.getValue() >= HIVE_CLI_SERVICE_PROTOCOL_V6.getValue()) { return new ColumnBasedSet(results); } diff --git service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java index dfb7faa..efa134a 100644 --- service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java +++ service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java @@ -38,11 +38,13 @@ import org.apache.hive.service.auth.HiveAuthFactory; import org.apache.hive.service.auth.TSetIpAddressProcessor; import org.apache.hive.service.cli.CLIService; +import org.apache.hive.service.cli.EncodedColumnBasedSet; import org.apache.hive.service.cli.FetchOrientation; import org.apache.hive.service.cli.FetchType; import org.apache.hive.service.cli.GetInfoType; import org.apache.hive.service.cli.GetInfoValue; import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.operation.Operation; import org.apache.hive.service.cli.OperationHandle; import org.apache.hive.service.cli.OperationStatus; import org.apache.hive.service.cli.RowSet; @@ -673,11 +675,17 @@ public TGetResultSetMetadataResp GetResultSetMetadata(TGetResultSetMetadataReq r public TFetchResultsResp FetchResults(TFetchResultsReq req) throws TException { TFetchResultsResp resp = new TFetchResultsResp(); try { - RowSet rowSet = cliService.fetchResults( - new OperationHandle(req.getOperationHandle()), - FetchOrientation.getFetchOrientation(req.getOrientation()), - req.getMaxRows(), - FetchType.getFetchType(req.getFetchType())); + OperationHandle opHandle = new OperationHandle(req.getOperationHandle()); + RowSet rowSet = cliService.fetchResults(opHandle, + FetchOrientation.getFetchOrientation(req.getOrientation()), + req.getMaxRows(), + FetchType.getFetchType(req.getFetchType())); + if (rowSet instanceof EncodedColumnBasedSet) { + Operation operation = this.cliService.getSessionManager().getOperationManager() + .getOperation(opHandle); + HiveConf sessionConf = operation.getParentSession().getHiveConf(); + ((EncodedColumnBasedSet) rowSet).setConf(sessionConf); + } resp.setResults(rowSet.toTRowSet()); resp.setHasMoreRows(false); resp.setStatus(OK_STATUS);