You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by br...@apache.org on 2014/11/02 22:08:27 UTC

svn commit: r1636192 - in /hive/trunk: common/src/java/org/apache/hadoop/hive/conf/HiveConf.java metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java service/src/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java

Author: brock
Date: Sun Nov  2 21:08:27 2014
New Revision: 1636192

URL: http://svn.apache.org/r1636192
Log:
HIVE-8680 - Set Max Message for Binary Thrift endpoints (Brock reviewed by Szehon)

Modified:
    hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java

Modified: hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
URL: http://svn.apache.org/viewvc/hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java?rev=1636192&r1=1636191&r2=1636192&view=diff
==============================================================================
--- hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (original)
+++ hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java Sun Nov  2 21:08:27 2014
@@ -384,6 +384,8 @@ public class HiveConf extends Configurat
         "the connection URL, before the next metastore query that accesses the\n" +
         "datastore. Once reloaded, this value is reset to false. Used for\n" +
         "testing only."),
+    METASTORESERVERMAXMESSAGESIZE("hive.metastore.server.max.message.size", 100*1024*1024,
+        "Maximum message size in bytes a HMS will accept."),
     METASTORESERVERMINTHREADS("hive.metastore.server.min.threads", 200,
         "Minimum number of worker threads in the Thrift server's pool."),
     METASTORESERVERMAXTHREADS("hive.metastore.server.max.threads", 1000,
@@ -1612,6 +1614,8 @@ public class HiveConf extends Configurat
         "Port number of HiveServer2 Thrift interface when hive.server2.transport.mode is 'http'."),
     HIVE_SERVER2_THRIFT_HTTP_PATH("hive.server2.thrift.http.path", "cliservice",
         "Path component of URL endpoint when in HTTP mode."),
+    HIVE_SERVER2_THRIFT_MAX_MESSAGE_SIZE("hive.server2.thrift.max.message.size", 100*1024*1024,
+        "Maximum message size in bytes a HS2 server will accept."),
     HIVE_SERVER2_THRIFT_HTTP_MIN_WORKER_THREADS("hive.server2.thrift.http.min.worker.threads", 5,
         "Minimum number of worker threads when in HTTP mode."),
     HIVE_SERVER2_THRIFT_HTTP_MAX_WORKER_THREADS("hive.server2.thrift.http.max.worker.threads", 500,

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java?rev=1636192&r1=1636191&r2=1636192&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java Sun Nov  2 21:08:27 2014
@@ -5773,6 +5773,7 @@ public class HiveMetaStore extends Thrif
       // Server will create new threads up to max as necessary. After an idle
       // period, it will destroy threads to keep the number of threads in the
       // pool to min.
+      int maxMessageSize = conf.getIntVar(HiveConf.ConfVars.METASTORESERVERMAXMESSAGESIZE);
       int minWorkerThreads = conf.getIntVar(HiveConf.ConfVars.METASTORESERVERMINTHREADS);
       int maxWorkerThreads = conf.getIntVar(HiveConf.ConfVars.METASTORESERVERMAXTHREADS);
       boolean tcpKeepAlive = conf.getBoolVar(HiveConf.ConfVars.METASTORE_TCP_KEEP_ALIVE);
@@ -5824,6 +5825,7 @@ public class HiveMetaStore extends Thrif
           .processor(processor)
           .transportFactory(transFactory)
           .protocolFactory(new TBinaryProtocol.Factory())
+          .inputProtocolFactory(new TBinaryProtocol.Factory(true, true, maxMessageSize))
           .minWorkerThreads(minWorkerThreads)
           .maxWorkerThreads(maxWorkerThreads);
 

Modified: hive/trunk/service/src/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java?rev=1636192&r1=1636191&r2=1636192&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java Sun Nov  2 21:08:27 2014
@@ -71,9 +71,12 @@ public class ThriftBinaryCLIService exte
       }
 
       // Server args
+      int maxMessageSize = hiveConf.getIntVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_MAX_MESSAGE_SIZE);
       TThreadPoolServer.Args sargs = new TThreadPoolServer.Args(serverSocket)
           .processorFactory(processorFactory).transportFactory(transportFactory)
-          .protocolFactory(new TBinaryProtocol.Factory()).executorService(executorService);
+          .protocolFactory(new TBinaryProtocol.Factory())
+          .inputProtocolFactory(new TBinaryProtocol.Factory(true, true, maxMessageSize))
+          .executorService(executorService);
 
       // TCP Server
       server = new TThreadPoolServer(sargs);