diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java index f4d1c376d3fc..db330fa8758a 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -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" + diff --git a/service/src/java/org/apache/hive/service/cli/thrift/RetryingThriftCLIServiceClient.java b/service/src/java/org/apache/hive/service/cli/thrift/RetryingThriftCLIServiceClient.java index 9079c652d0a8..513836b23c84 100644 --- a/service/src/java/org/apache/hive/service/cli/thrift/RetryingThriftCLIServiceClient.java +++ b/service/src/java/org/apache/hive/service/cli/thrift/RetryingThriftCLIServiceClient.java @@ -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, diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java index d3e3479899a3..2f29391bb315 100644 --- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java +++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java @@ -610,7 +610,7 @@ public void renamePartition(String catName, String dbname, String tableName, Lis } private 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", diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java index 62ea1a63eaec..08ff1d6c394f 100644 --- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java +++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java @@ -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", "",