Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
8 changes: 4 additions & 4 deletions common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
Original file line number Diff line number Diff line change
Expand Up @@ -2924,10 +2924,10 @@ public static enum ConfVars {
HIVE_STATS_MAX_NUM_STATS("hive.stats.max.num.stats", (long) 10000,
"When the number of stats to be updated is huge, this value is used to control the number of \n" +
" stats to be sent to HMS for update."),
HIVE_THRIFT_CLIENT_MAX_MESSAGE_SIZE("hive.thrift.client.max.message.size", "1gb",
new SizeValidator(-1L, true, (long) Integer.MAX_VALUE, true),
"Thrift client configuration for max message size. 0 or -1 will use the default defined in the Thrift " +
"library. The upper limit is 2147483648 bytes (or 2gb)."),
HIVE_THRIFT_CLIENT_MAX_MESSAGE_SIZE("hive.thrift.client.max.message.size", (2*1024*1024*1024L)-1L,
new SizeValidator(-1L, true, (long) Integer.MAX_VALUE, true),
"Thrift client configuration for max message size. 0 or -1 will use the default defined in the Thrift " +
"library. The upper limit is 2147483647 bytes."),
// Concurrency
HIVE_SUPPORT_CONCURRENCY("hive.support.concurrency", false,
"Whether Hive supports concurrency control or not. \n" +
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -310,8 +310,8 @@ protected synchronized TTransport connect(HiveConf conf) throws HiveSQLException

String host = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST);
int port = conf.getIntVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_PORT);
int maxThriftMessageSize = (int) conf.getSizeVar(HiveConf.ConfVars.HIVE_THRIFT_CLIENT_MAX_MESSAGE_SIZE);
LOG.info("Connecting to " + host + ":" + port);
int maxThriftMessageSize = (int) Math.min(conf.getSizeVar(HiveConf.ConfVars.HIVE_THRIFT_CLIENT_MAX_MESSAGE_SIZE),Integer.MAX_VALUE);
LOG.info("Connecting to " + host + ":" + port+ " using a thrift max message of size: "+ maxThriftMessageSize);

transport = HiveAuthUtils.getSocketTransport(host, port, 0, maxThriftMessageSize);
((TSocket) transport).setTimeout((int) conf.getTimeVar(HiveConf.ConfVars.SERVER_READ_SOCKET_TIMEOUT,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -610,7 +610,7 @@ public void renamePartition(String catName, String dbname, String tableName, Lis
}

private <T extends TTransport> T configureThriftMaxMessageSize(T transport) {
int maxThriftMessageSize = (int) MetastoreConf.getSizeVar(conf, ConfVars.THRIFT_METASTORE_CLIENT_MAX_MESSAGE_SIZE);
int maxThriftMessageSize = (int) Math.min(MetastoreConf.getSizeVar(conf, ConfVars.THRIFT_METASTORE_CLIENT_MAX_MESSAGE_SIZE),Integer.MAX_VALUE);
if (maxThriftMessageSize > 0) {
if (transport.getConfiguration() == null) {
LOG.warn("TTransport {} is returning a null Configuration, Thrift max message size is not getting configured",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1443,9 +1443,9 @@ public enum ConfVars {
" corresponding service discovery servers e.g. a zookeeper. Otherwise they are " +
"used as URIs for remote metastore."),
THRIFT_METASTORE_CLIENT_MAX_MESSAGE_SIZE("metastore.thrift.client.max.message.size",
"hive.thrift.client.max.message.size", "1gb", new SizeValidator(-1L, true, (long) Integer.MAX_VALUE, true),
"Thrift client configuration for max message size. 0 or -1 will use the default defined in the Thrift " +
"library. The upper limit is 2147483648 bytes (or 2gb)."),
"hive.thrift.client.max.message.size", (2*1024*1024*1024L)-1L, new SizeValidator(-1L, true, (long) Integer.MAX_VALUE, true),
"Thrift client configuration for max message size. 0 or -1 will use the default defined in the Thrift " +
"library. The upper limit is 2147483647 bytes"),
THRIFT_SERVICE_DISCOVERY_MODE("metastore.service.discovery.mode",
"hive.metastore.service.discovery.mode",
"",
Expand Down