[ 
https://issues.apache.org/jira/browse/HIVE-26633?focusedWorklogId=818309&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-818309
 ]

ASF GitHub Bot logged work on HIVE-26633:
-----------------------------------------

                Author: ASF GitHub Bot
            Created on: 19/Oct/22 03:04
            Start Date: 19/Oct/22 03:04
    Worklog Time Spent: 10m 
      Work Description: jfsii commented on code in PR #3674:
URL: https://github.com/apache/hive/pull/3674#discussion_r998898238


##########
common/src/java/org/apache/hadoop/hive/common/auth/HiveAuthUtils.java:
##########
@@ -50,45 +49,108 @@
 public class HiveAuthUtils {
   private static final Logger LOG = 
LoggerFactory.getLogger(HiveAuthUtils.class);
 
+  /**
+   * Configure the provided T transport's max message size.
+   * @param transport Transport to configure maxMessage for
+   * @param maxMessageSize Maximum allowed message size in bytes, less than or 
equal to 0 means use the Thrift library
+   *                       default.
+   * @return The passed in T transport configured with desired max message 
size. The same object passed in is returned.
+   */
+  public static <T extends TTransport> T configureThriftMaxMessageSize(T 
transport, int maxMessageSize) {
+    if (maxMessageSize > 0) {
+      if (transport.getConfiguration() == null) {
+        LOG.warn("TTransport {} is returning a null Configuration, Thrift max 
message size is not getting configured",
+            transport.getClass().getName());
+        return transport;
+      }
+      transport.getConfiguration().setMaxMessageSize(maxMessageSize);
+    }
+    return transport;
+  }
+
+  /**
+   * Create a TSocket for the provided host and port with specified 
loginTimeout. Thrift maxMessageSize
+   * will default to Thrift library default.
+   * @param host Host to connect to.
+   * @param port Port to connect to.
+   * @param loginTimeout Socket timeout (0 means no timeout).
+   * @return TTransport TSocket for host/port.
+   */
   public static TTransport getSocketTransport(String host, int port, int 
loginTimeout) throws TTransportException {
-    return new TSocket(new TConfiguration(),host, port, loginTimeout);
+    return getSocketTransport(host, port, loginTimeout, -1);
+  }
+
+  /**
+   * Create a TSocket for the provided host and port with specified 
loginTimeout and maxMessageSize.
+   * will default to Thrift library default.
+   * @param host Host to connect to.
+   * @param port Port to connect to.
+   * @param loginTimeout Socket timeout (0 means no timeout).
+   * @param maxMessageSize Size in bytes for max allowable Thrift message 
size, less than or equal to 0
+   *                       results in using the Thrift library default.
+   * @return TTransport TSocket for host/port
+   */
+  public static TTransport getSocketTransport(String host, int port, int 
loginTimeout, int maxMessageSize)
+      throws TTransportException {
+    TSocket tSocket = new TSocket(host, port, loginTimeout);
+    return configureThriftMaxMessageSize(tSocket, maxMessageSize);
+  }
+
+  public static TTransport getSSLSocket(String host, int port, int 
loginTimeout, TSSLTransportParameters params,
+      int maxMessageSize) throws TTransportException {
+    // The underlying SSLSocket object is bound to host:port with the given 
SO_TIMEOUT and
+    // SSLContext created with the given params
+    TSocket tSSLSocket = null;
+    if (params != null) {
+      tSSLSocket = TSSLTransportFactory.getClientSocket(host, port, 
loginTimeout, params);
+    } else {
+      tSSLSocket = TSSLTransportFactory.getClientSocket(host, port, 
loginTimeout);
+    }
+    configureThriftMaxMessageSize(tSSLSocket, maxMessageSize);

Review Comment:
   I was being overly paranoid. Looking closer at getSSLSocketWithHttps, I see 
it doesn't even end up using the TSocket. It grabs the SSLSocket and re-wraps 
it with TSocket. When I originally read the code I thought it was a TSocket 
wrapped by a TSocket which I didn't know which conf would take (but I was 
wrong).





Issue Time Tracking
-------------------

    Worklog Id:     (was: 818309)
    Time Spent: 3h 40m  (was: 3.5h)

> Make thrift max message size configurable
> -----------------------------------------
>
>                 Key: HIVE-26633
>                 URL: https://issues.apache.org/jira/browse/HIVE-26633
>             Project: Hive
>          Issue Type: Bug
>          Components: HiveServer2
>    Affects Versions: 4.0.0-alpha-2
>            Reporter: John Sherman
>            Assignee: John Sherman
>            Priority: Major
>              Labels: pull-request-available
>          Time Spent: 3h 40m
>  Remaining Estimate: 0h
>
> Since thrift >= 0.14, thrift now enforces max message sizes through a 
> TConfiguration object as described here:
> [https://github.com/apache/thrift/blob/master/doc/specs/thrift-tconfiguration.md]
> By default MaxMessageSize gets set to 100MB.
> As a result it is possible for HMS clients not to be able to retrieve certain 
> metadata for tables with a large amount of partitions or other metadata.
> For example on a cluster configured with kerberos between hs2 and hms, 
> querying a large table (10k partitions, 200 columns with names of 200 
> characters) results in this backtrace:
> {code:java}
> org.apache.thrift.transport.TTransportException: MaxMessageSize reached
> at 
> org.apache.thrift.transport.TEndpointTransport.countConsumedMessageBytes(TEndpointTransport.java:96)
>  
> at 
> org.apache.thrift.transport.TMemoryInputTransport.read(TMemoryInputTransport.java:97)
>  
> at org.apache.thrift.transport.TSaslTransport.read(TSaslTransport.java:390) 
> at 
> org.apache.thrift.transport.TSaslClientTransport.read(TSaslClientTransport.java:39)
>  
> at org.apache.thrift.transport.TTransport.readAll(TTransport.java:109) 
> at 
> org.apache.hadoop.hive.metastore.security.TFilterTransport.readAll(TFilterTransport.java:63)
>  
> at 
> org.apache.thrift.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:464) 
> at 
> org.apache.thrift.protocol.TBinaryProtocol.readByte(TBinaryProtocol.java:329) 
> at 
> org.apache.thrift.protocol.TBinaryProtocol.readFieldBegin(TBinaryProtocol.java:273)
>  
> at 
> org.apache.hadoop.hive.metastore.api.FieldSchema$FieldSchemaStandardScheme.read(FieldSchema.java:461)
>  
> at 
> org.apache.hadoop.hive.metastore.api.FieldSchema$FieldSchemaStandardScheme.read(FieldSchema.java:454)
>  
> at 
> org.apache.hadoop.hive.metastore.api.FieldSchema.read(FieldSchema.java:388) 
> at 
> org.apache.hadoop.hive.metastore.api.StorageDescriptor$StorageDescriptorStandardScheme.read(StorageDescriptor.java:1269)
>  
> at 
> org.apache.hadoop.hive.metastore.api.StorageDescriptor$StorageDescriptorStandardScheme.read(StorageDescriptor.java:1248)
>  
> at 
> org.apache.hadoop.hive.metastore.api.StorageDescriptor.read(StorageDescriptor.java:1110)
>  
> at 
> org.apache.hadoop.hive.metastore.api.Partition$PartitionStandardScheme.read(Partition.java:1270)
>  
> at 
> org.apache.hadoop.hive.metastore.api.Partition$PartitionStandardScheme.read(Partition.java:1205)
>  
> at org.apache.hadoop.hive.metastore.api.Partition.read(Partition.java:1062) 
> at 
> org.apache.hadoop.hive.metastore.api.PartitionsByExprResult$PartitionsByExprResultStandardScheme.read(PartitionsByExprResult.java:420)
>  
> at 
> org.apache.hadoop.hive.metastore.api.PartitionsByExprResult$PartitionsByExprResultStandardScheme.read(PartitionsByExprResult.java:399)
>  
> at 
> org.apache.hadoop.hive.metastore.api.PartitionsByExprResult.read(PartitionsByExprResult.java:335)
>  
> at 
> org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$get_partitions_by_expr_result$get_partitions_by_expr_resultStandardScheme.read(ThriftHiveMetastore.java)
>   {code}
> Making this configurable (and defaulting to a higher value) would allow these 
> tables to still be accessible.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to