From 87c7c4626457fbff377ae26de7ee530919e931d3 Mon Sep 17 00:00:00 2001 From: Vlad Rozov Date: Fri, 7 Mar 2025 16:28:03 +0000 Subject: [PATCH 01/19] [WIP][SPARK-51348][BUILD][SQL] Upgrade Hive to 4.0 --- pom.xml | 23 +++++- sql/core/pom.xml | 6 +- .../execution/datasources/DaysWritable.scala | 15 ++-- .../datasources/orc/OrcShimUtils.scala | 6 +- .../hive/service/auth/HiveAuthFactory.java | 17 ++-- .../hive/service/auth/HttpAuthUtils.java | 6 +- .../hive/service/auth/KerberosSaslHelper.java | 8 +- .../operation/ExecuteStatementOperation.java | 6 +- .../cli/operation/GetCatalogsOperation.java | 4 +- .../cli/operation/GetColumnsOperation.java | 8 +- .../operation/GetCrossReferenceOperation.java | 4 +- .../cli/operation/GetFunctionsOperation.java | 4 +- .../operation/GetPrimaryKeysOperation.java | 4 +- .../cli/operation/GetSchemasOperation.java | 4 +- .../cli/operation/GetTableTypesOperation.java | 4 +- .../cli/operation/GetTablesOperation.java | 4 +- .../cli/operation/GetTypeInfoOperation.java | 4 +- .../cli/operation/LogDivertAppender.java | 16 +++- .../cli/operation/MetadataOperation.java | 4 +- .../hive/service/cli/operation/Operation.java | 36 +++++---- .../cli/operation/OperationManager.java | 42 +++++++--- .../service/cli/session/HiveSessionImpl.java | 5 +- .../cli/session/HiveSessionImplwithUGI.java | 15 +++- .../cli/thrift/ThriftBinaryCLIService.java | 7 +- .../server/ThreadWithGarbageCleanup.java | 4 +- .../SparkExecuteStatementOperation.scala | 6 +- .../SparkGetCatalogsOperation.scala | 7 +- .../SparkGetColumnsOperation.scala | 6 +- .../SparkGetFunctionsOperation.scala | 6 +- .../SparkGetSchemasOperation.scala | 5 +- .../SparkGetTableTypesOperation.scala | 7 +- .../SparkGetTablesOperation.scala | 6 +- .../SparkGetTypeInfoOperation.scala | 7 +- .../hive/thriftserver/SparkSQLCLIDriver.scala | 81 +++++++++++-------- .../hive/thriftserver/SparkSQLDriver.scala | 11 +-- .../server/SparkSQLOperationManager.scala | 20 ++--- .../GetCatalogsOperationMock.scala | 6 +- .../thriftserver/HiveSessionImplSuite.scala | 2 +- .../thriftserver/SharedThriftServer.scala | 2 +- .../SparkExecuteStatementOperationSuite.scala | 6 +- ...arkThriftServerProtocolVersionsSuite.scala | 6 +- sql/hive/pom.xml | 4 + .../spark/sql/hive/HiveDateTimeUtils.scala | 35 ++++++++ .../spark/sql/hive/HiveInspectors.scala | 16 ++-- .../org/apache/spark/sql/hive/HiveUtils.scala | 14 ++-- .../apache/spark/sql/hive/TableReader.scala | 32 +++----- .../sql/hive/client/HiveClientImpl.scala | 49 ++--------- .../spark/sql/hive/client/HiveShim.scala | 35 ++------ .../sql/hive/execution/HiveFileFormat.scala | 10 +-- .../HiveScriptTransformationExec.scala | 2 +- .../hive/execution/HiveTableScanExec.scala | 3 +- .../sql/hive/execution/HiveTempPath.scala | 2 +- .../hive/execution/InsertIntoHiveTable.scala | 4 +- .../spark/sql/hive/orc/OrcFileFormat.scala | 2 +- .../sql/hive/execution/HiveQuerySuite.scala | 4 +- .../sql/hive/execution/HiveUDFSuite.scala | 5 +- 56 files changed, 359 insertions(+), 298 deletions(-) create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveDateTimeUtils.scala diff --git a/pom.xml b/pom.xml index 96e67c7a2697..af15e013a139 100644 --- a/pom.xml +++ b/pom.xml @@ -134,7 +134,7 @@ org.apache.hive core - 2.3.10 + 4.0.1 3.9.1 @@ -1807,6 +1807,10 @@ ${hive.group} hive-storage-api + + org.apache.tez + tez-api + @@ -1959,6 +1963,14 @@ net.hydromatic aggdesigner-algorithm + + org.apache.atlas + atlas-intg + + + org.apache.hadoop + hadoop-yarn-registry + @@ -2262,6 +2274,10 @@ ${hive.group} hive-serde + + org.apache.hadoop + hadoop-common + org.slf4j slf4j-api @@ -2305,6 +2321,11 @@ + + org.apache.hive + hive-udf + ${hive.version} + org.apache.orc diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 39d8c3995441..9406d8026a91 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -123,7 +123,11 @@ ${hive.group} - hive-storage-api + hive-common + + + ${hive.group} + hive-serde org.apache.parquet diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DaysWritable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DaysWritable.scala index a04c2fcbbac1..93be706c7868 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DaysWritable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DaysWritable.scala @@ -18,9 +18,9 @@ package org.apache.spark.sql.execution.datasources import java.io.{DataInput, DataOutput, IOException} -import java.sql.Date -import org.apache.hadoop.hive.serde2.io.DateWritable +import org.apache.hadoop.hive.common.`type`.Date +import org.apache.hadoop.hive.serde2.io.DateWritableV2 import org.apache.hadoop.io.WritableUtils import org.apache.spark.sql.catalyst.util.RebaseDateTime.{rebaseGregorianToJulianDays, rebaseJulianToGregorianDays} @@ -38,16 +38,16 @@ import org.apache.spark.sql.catalyst.util.RebaseDateTime.{rebaseGregorianToJulia class DaysWritable( var gregorianDays: Int, var julianDays: Int) - extends DateWritable { + extends DateWritableV2 { def this() = this(0, 0) def this(gregorianDays: Int) = this(gregorianDays, rebaseGregorianToJulianDays(gregorianDays)) - def this(dateWritable: DateWritable) = { + def this(dateWritable: DateWritableV2) = { this( gregorianDays = dateWritable match { case daysWritable: DaysWritable => daysWritable.gregorianDays - case dateWritable: DateWritable => + case dateWritable: DateWritableV2 => rebaseJulianToGregorianDays(dateWritable.getDays) }, julianDays = dateWritable.getDays) @@ -55,10 +55,7 @@ class DaysWritable( override def getDays: Int = julianDays override def get: Date = { - new Date(DateWritable.daysToMillis(julianDays)) - } - override def get(doesTimeMatter: Boolean): Date = { - new Date(DateWritable.daysToMillis(julianDays, doesTimeMatter)) + Date.ofEpochMilli(DateWritableV2.daysToMillis(julianDays)) } override def set(d: Int): Unit = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala index 60c5b7a266c5..ec33af76afbd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala @@ -21,7 +21,7 @@ import org.apache.hadoop.hive.common.`type`.HiveDecimal import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch import org.apache.hadoop.hive.ql.io.sarg.{SearchArgument => OrcSearchArgument} import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf.{Operator => OrcOperator} -import org.apache.hadoop.hive.serde2.io.{DateWritable, HiveDecimalWritable} +import org.apache.hadoop.hive.serde2.io.{DateWritableV2, HiveDecimalWritable} import org.apache.spark.sql.catalyst.expressions.SpecializedGetters import org.apache.spark.sql.execution.datasources.DaysWritable @@ -38,7 +38,7 @@ private[sql] object OrcShimUtils { private[sql] type SearchArgument = OrcSearchArgument def getGregorianDays(value: Any): Int = { - new DaysWritable(value.asInstanceOf[DateWritable]).gregorianDays + new DaysWritable(value.asInstanceOf[DateWritableV2]).gregorianDays } def getDecimal(value: Any): Decimal = { @@ -46,7 +46,7 @@ private[sql] object OrcShimUtils { Decimal(decimal.bigDecimalValue, decimal.precision(), decimal.scale()) } - def getDateWritable(reuseObj: Boolean): (SpecializedGetters, Int) => DateWritable = { + def getDateWritable(reuseObj: Boolean): (SpecializedGetters, Int) => DateWritableV2 = { if (reuseObj) { val result = new DaysWritable() (getter, ordinal) => diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java index da4c170f93fd..6dd67be7de11 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java @@ -30,10 +30,10 @@ import org.apache.hadoop.hive.ql.metadata.Hive; import org.apache.hadoop.hive.shims.HadoopShims.KerberosNameShim; import org.apache.hadoop.hive.shims.ShimLoader; -import org.apache.hadoop.hive.thrift.DBTokenStore; -import org.apache.hadoop.hive.thrift.HiveDelegationTokenManager; -import org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge; -import org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge.Server.ServerMode; +import org.apache.hadoop.hive.metastore.security.DBTokenStore; +import org.apache.hadoop.hive.metastore.security.MetastoreDelegationTokenManager; +import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge; +import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge.Server.ServerMode; import org.apache.hadoop.security.SecurityUtil; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.authorize.ProxyUsers; @@ -80,7 +80,7 @@ public String getAuthName() { private String authTypeStr; private final String transportMode; private final HiveConf conf; - private HiveDelegationTokenManager delegationTokenManager = null; + private MetastoreDelegationTokenManager delegationTokenManager = null; public static final String HS2_PROXY_USER = "hive.server2.proxy.user"; public static final String HS2_CLIENT_TOKEN = "hiveserver2ClientToken"; @@ -114,18 +114,19 @@ public HiveAuthFactory(HiveConf conf) throws TTransportException, IOException { authTypeStr = AuthTypes.NONE.getAuthName(); } if (authTypeStr.equalsIgnoreCase(AuthTypes.KERBEROS.getAuthName())) { - String principal = conf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_PRINCIPAL); String keytab = conf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_KEYTAB); + String principal = conf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_PRINCIPAL); + String client = conf.getVar(ConfVars.HIVE_SERVER2_CLIENT_KERBEROS_PRINCIPAL); if (needUgiLogin(UserGroupInformation.getCurrentUser(), SecurityUtil.getServerPrincipal(principal, "0.0.0.0"), keytab)) { - saslServer = ShimLoader.getHadoopThriftAuthBridge().createServer(keytab, principal); + saslServer = HadoopThriftAuthBridge.getBridge().createServer(keytab, principal, client); } else { // Using the default constructor to avoid unnecessary UGI login. saslServer = new HadoopThriftAuthBridge.Server(); } // start delegation token manager - delegationTokenManager = new HiveDelegationTokenManager(); + delegationTokenManager = new MetastoreDelegationTokenManager(); try { // rawStore is only necessary for DBTokenStore Object rawStore = null; diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java index a76bcc9b873e..489608de72f3 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java @@ -31,6 +31,7 @@ import javax.security.auth.Subject; import org.apache.commons.codec.binary.Base64; +import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge; import org.apache.hadoop.hive.shims.ShimLoader; import org.apache.hadoop.security.UserGroupInformation; import org.apache.http.protocol.BasicHttpContext; @@ -68,8 +69,7 @@ public final class HttpAuthUtils { */ public static String getKerberosServiceTicket(String principal, String host, String serverHttpUrl, boolean assumeSubject) throws Exception { - String serverPrincipal = - ShimLoader.getHadoopThriftAuthBridge().getServerPrincipal(principal, host); + String serverPrincipal = HadoopThriftAuthBridge.getBridge().getServerPrincipal(principal, host); if (assumeSubject) { // With this option, we're assuming that the external application, // using the JDBC driver has done a JAAS kerberos login already @@ -82,7 +82,7 @@ public static String getKerberosServiceTicket(String principal, String host, } else { // JAAS login from ticket cache to setup the client UserGroupInformation UserGroupInformation clientUGI = - ShimLoader.getHadoopThriftAuthBridge().getCurrentUGIWithConf("kerberos"); + HadoopThriftAuthBridge.getBridge().getCurrentUGIWithConf("kerberos"); return clientUGI.doAs(new HttpKerberosClientAction(serverPrincipal, serverHttpUrl)); } } diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java index ef91f94eeec2..4b05d72530dd 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java @@ -21,8 +21,8 @@ import javax.security.sasl.SaslException; import org.apache.hadoop.hive.shims.ShimLoader; -import org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge; -import org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge.Server; +import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge; +import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge.Server; import org.apache.hive.service.cli.thrift.ThriftCLIService; import org.apache.hive.service.rpc.thrift.TCLIService; import org.apache.hive.service.rpc.thrift.TCLIService.Iface; @@ -52,7 +52,7 @@ public static TTransport getKerberosTransport(String principal, String host, return createSubjectAssumedTransport(principal, underlyingTransport, saslProps); } else { HadoopThriftAuthBridge.Client authBridge = - ShimLoader.getHadoopThriftAuthBridge().createClientWithConf("kerberos"); + HadoopThriftAuthBridge.getBridge().createClientWithConf("kerberos"); return authBridge.createClientTransport(principal, host, "KERBEROS", null, underlyingTransport, saslProps); } @@ -77,7 +77,7 @@ public static TTransport createSubjectAssumedTransport(String principal, public static TTransport getTokenTransport(String tokenStr, String host, TTransport underlyingTransport, Map saslProps) throws SaslException { HadoopThriftAuthBridge.Client authBridge = - ShimLoader.getHadoopThriftAuthBridge().createClientWithConf("kerberos"); + HadoopThriftAuthBridge.getBridge().createClientWithConf("kerberos"); try { return authBridge.createClientTransport(null, host, "DIGEST", tokenStr, underlyingTransport, diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java index 815a369b6b23..4c4a3f60e5e7 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java @@ -25,9 +25,9 @@ public abstract class ExecuteStatementOperation extends Operation { protected String statement = null; - public ExecuteStatementOperation(HiveSession parentSession, String statement, + public ExecuteStatementOperation(HiveSession parentSession, OperationManager operationManager, String statement, Map confOverlay, boolean runInBackground) { - super(parentSession, confOverlay, OperationType.EXECUTE_STATEMENT, runInBackground); + super(parentSession, operationManager, confOverlay, OperationType.EXECUTE_STATEMENT, runInBackground); this.statement = statement; } @@ -43,7 +43,7 @@ protected void registerCurrentOperationLog() { isOperationLogEnabled = false; return; } - OperationLog.setCurrentOperationLog(operationLog); + operationManager.setCurrentOperationLog(operationLog, operationLogFile); } } } diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java index ef4bbb45e8f4..e597d328635a 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java @@ -33,8 +33,8 @@ public class GetCatalogsOperation extends MetadataOperation { protected final RowSet rowSet; - protected GetCatalogsOperation(HiveSession parentSession) { - super(parentSession, OperationType.GET_CATALOGS); + protected GetCatalogsOperation(HiveSession parentSession, OperationManager operationManager) { + super(parentSession, operationManager, OperationType.GET_CATALOGS); rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion(), false); } diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java index 250adc51f81e..1cfee892adac 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java @@ -33,7 +33,7 @@ import org.apache.hadoop.hive.metastore.api.PrimaryKeysRequest; import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey; import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.ql.metadata.TableIterable; +import org.apache.hadoop.hive.metastore.TableIterable; import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject; import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivilegeObjectType; @@ -119,9 +119,9 @@ public class GetColumnsOperation extends MetadataOperation { protected final RowSet rowSet; - protected GetColumnsOperation(HiveSession parentSession, String catalogName, String schemaName, - String tableName, String columnName) { - super(parentSession, OperationType.GET_COLUMNS); + protected GetColumnsOperation(HiveSession parentSession, OperationManager operationManager, + String catalogName, String schemaName, String tableName, String columnName) { + super(parentSession, operationManager, OperationType.GET_COLUMNS); this.catalogName = catalogName; this.schemaName = schemaName; this.tableName = tableName; diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetCrossReferenceOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetCrossReferenceOperation.java index 3a29859a2074..b21ba0d09271 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetCrossReferenceOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetCrossReferenceOperation.java @@ -99,10 +99,10 @@ public class GetCrossReferenceOperation extends MetadataOperation { private final String foreignTableName; private final RowSet rowSet; - public GetCrossReferenceOperation(HiveSession parentSession, + public GetCrossReferenceOperation(HiveSession parentSession, OperationManager operationManager, String parentCatalogName, String parentSchemaName, String parentTableName, String foreignCatalog, String foreignSchema, String foreignTable) { - super(parentSession, OperationType.GET_FUNCTIONS); + super(parentSession, operationManager, OperationType.GET_FUNCTIONS); this.parentCatalogName = parentCatalogName; this.parentSchemaName = parentSchemaName; this.parentTableName = parentTableName; diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java index 3f02f753bf87..a78cd6fb08c4 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java @@ -66,9 +66,9 @@ public class GetFunctionsOperation extends MetadataOperation { protected final RowSet rowSet; - public GetFunctionsOperation(HiveSession parentSession, + public GetFunctionsOperation(HiveSession parentSession, OperationManager operationManager, String catalogName, String schemaName, String functionName) { - super(parentSession, OperationType.GET_FUNCTIONS); + super(parentSession, operationManager, OperationType.GET_FUNCTIONS); this.catalogName = catalogName; this.schemaName = schemaName; this.functionName = functionName; diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetPrimaryKeysOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetPrimaryKeysOperation.java index 927328342974..9339865f29f6 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetPrimaryKeysOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetPrimaryKeysOperation.java @@ -61,9 +61,9 @@ public class GetPrimaryKeysOperation extends MetadataOperation { private final RowSet rowSet; - public GetPrimaryKeysOperation(HiveSession parentSession, + public GetPrimaryKeysOperation(HiveSession parentSession, OperationManager operationManager, String catalogName, String schemaName, String tableName) { - super(parentSession, OperationType.GET_FUNCTIONS); + super(parentSession, operationManager, OperationType.GET_FUNCTIONS); this.catalogName = catalogName; this.schemaName = schemaName; this.tableName = tableName; diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java index 865e264bd5f4..a71908519322 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java @@ -44,9 +44,9 @@ public class GetSchemasOperation extends MetadataOperation { protected RowSet rowSet; - protected GetSchemasOperation(HiveSession parentSession, + protected GetSchemasOperation(HiveSession parentSession, OperationManager operationManager, String catalogName, String schemaName) { - super(parentSession, OperationType.GET_SCHEMAS); + super(parentSession, operationManager, OperationType.GET_SCHEMAS); this.catalogName = catalogName; this.schemaName = schemaName; this.rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion(), false); diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java index b75eaec5ff65..9009da795d1d 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java @@ -43,8 +43,8 @@ public class GetTableTypesOperation extends MetadataOperation { protected final RowSet rowSet; private final TableTypeMapping tableTypeMapping; - protected GetTableTypesOperation(HiveSession parentSession) { - super(parentSession, OperationType.GET_TABLE_TYPES); + protected GetTableTypesOperation(HiveSession parentSession, OperationManager operationManager) { + super(parentSession, operationManager, OperationType.GET_TABLE_TYPES); String tableMappingStr = getParentSession().getHiveConf() .getVar(HiveConf.ConfVars.HIVE_SERVER2_TABLE_TYPE_MAPPING); tableTypeMapping = diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java index bd9f0814814f..c3bf9ffa0629 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java @@ -66,10 +66,10 @@ public class GetTablesOperation extends MetadataOperation { .addStringColumn("REF_GENERATION", "Specifies how values in SELF_REFERENCING_COL_NAME are created."); - protected GetTablesOperation(HiveSession parentSession, + protected GetTablesOperation(HiveSession parentSession, OperationManager operationManager, String catalogName, String schemaName, String tableName, List tableTypes) { - super(parentSession, OperationType.GET_TABLES); + super(parentSession, operationManager, OperationType.GET_TABLES); this.catalogName = catalogName; this.schemaName = schemaName; this.tableName = tableName; diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java index ad692d46edd2..9cdff0635334 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java @@ -76,8 +76,8 @@ public class GetTypeInfoOperation extends MetadataOperation { protected final RowSet rowSet; - protected GetTypeInfoOperation(HiveSession parentSession) { - super(parentSession, OperationType.GET_TYPE_INFO); + protected GetTypeInfoOperation(HiveSession parentSession, OperationManager operationManager) { + super(parentSession, operationManager, OperationType.GET_TYPE_INFO); rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion(), false); } diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/LogDivertAppender.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/LogDivertAppender.java index 32cc42f008bd..28023bea477b 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/LogDivertAppender.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/LogDivertAppender.java @@ -16,7 +16,11 @@ */ package org.apache.hive.service.cli.operation; +import java.io.BufferedOutputStream; import java.io.CharArrayWriter; +import java.io.FileNotFoundException; +import java.io.FileOutputStream; +import java.io.PrintStream; import java.io.Serializable; import java.util.Map; import java.util.regex.Pattern; @@ -180,7 +184,7 @@ public Result filter(org.apache.logging.log4j.core.Logger logger, Level level, M @Override public Result filter(LogEvent logEvent) { - OperationLog log = operationManager.getOperationLogByThread(); + OperationLog log = operationManager.getCurrentOperationLog(); boolean excludeMatches = (loggingMode == OperationLog.LoggingLevel.VERBOSE); if (log == null) { @@ -296,7 +300,7 @@ private LogDivertAppender(OperationManager operationManager, @Override public void append(LogEvent event) { - OperationLog log = operationManager.getOperationLogByThread(); + OperationLog log = operationManager.getCurrentOperationLog(); // Set current layout depending on the verbose/non-verbose mode. if (log != null) { @@ -318,6 +322,12 @@ public void append(LogEvent event) { LOG.debug(" ---+++=== Dropped log event from thread " + event.getThreadName()); return; } - log.writeOperationLog(logOutput); + try { + PrintStream out = + new PrintStream(new FileOutputStream(operationManager.getCurrentOperationLogFile())); + out.print(logOutput); + } catch (FileNotFoundException e) { + LOG.debug(" ---+++=== Dropped log event from thread " + event.getThreadName()); + } } } diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java index a818d1ecfd16..a111285df6f5 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java @@ -42,8 +42,8 @@ public abstract class MetadataOperation extends Operation { protected static TableSchema RESULT_SET_SCHEMA; private static final char SEARCH_STRING_ESCAPE = '\\'; - protected MetadataOperation(HiveSession parentSession, OperationType opType) { - super(parentSession, opType); + protected MetadataOperation(HiveSession parentSession, OperationManager operationManager, OperationType opType) { + super(parentSession, operationManager, opType); setHasResultSet(true); } diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/Operation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/Operation.java index b5cf8991a4cb..75ad4cd74211 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/Operation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/Operation.java @@ -21,11 +21,13 @@ import java.util.EnumSet; import java.util.HashMap; import java.util.Map; +import java.util.Objects; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.QueryState; +import org.apache.hadoop.hive.ql.processors.CommandProcessorException; import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse; import org.apache.hadoop.hive.ql.session.OperationLog; import org.apache.hive.service.cli.FetchOrientation; @@ -46,6 +48,7 @@ public abstract class Operation { protected final HiveSession parentSession; + protected final OperationManager operationManager; private OperationState state = OperationState.INITIALIZED; private final OperationHandle opHandle; private HiveConf configuration; @@ -56,6 +59,7 @@ public abstract class Operation { protected volatile HiveSQLException operationException; protected final boolean runAsync; protected volatile Future backgroundHandle; + protected File operationLogFile; protected OperationLog operationLog; protected boolean isOperationLogEnabled; protected Map confOverlay = new HashMap(); @@ -71,25 +75,29 @@ public abstract class Operation { FetchOrientation.FETCH_FIRST, FetchOrientation.FETCH_PRIOR); - protected Operation(HiveSession parentSession, OperationType opType) { - this(parentSession, null, opType); + protected Operation(HiveSession parentSession, OperationManager operationManager, OperationType opType) { + this(parentSession, operationManager, null, opType); } - protected Operation(HiveSession parentSession, Map confOverlay, + protected Operation(HiveSession parentSession, OperationManager operationManager, Map confOverlay, OperationType opType) { - this(parentSession, confOverlay, opType, false); + this(parentSession, operationManager, confOverlay, opType, false); } protected Operation(HiveSession parentSession, - Map confOverlay, OperationType opType, boolean runInBackground) { + OperationManager operationManager, Map confOverlay, OperationType opType, boolean runInBackground) { this.parentSession = parentSession; + this.operationManager = Objects.requireNonNull(operationManager); this.confOverlay = confOverlay; this.runAsync = runInBackground; this.opHandle = new OperationHandle(opType, parentSession.getProtocolVersion()); lastAccessTime = System.currentTimeMillis(); operationTimeout = HiveConf.getTimeVar(parentSession.getHiveConf(), HiveConf.ConfVars.HIVE_SERVER2_IDLE_OPERATION_TIMEOUT, TimeUnit.MILLISECONDS); - queryState = new QueryState(parentSession.getHiveConf(), confOverlay, runInBackground); + queryState = new QueryState.Builder() + .withHiveConf(parentSession.getHiveConf()) + .withConfOverlay(confOverlay) + .build(); } public Future getBackgroundHandle() { @@ -204,7 +212,7 @@ public boolean isFailed() { protected void createOperationLog() { if (parentSession.isOperationLogEnabled()) { - File operationLogFile = new File(parentSession.getOperationLogSessionDir(), + operationLogFile = new File(parentSession.getOperationLogSessionDir(), opHandle.getHandleIdentifier().toString()); isOperationLogEnabled = true; @@ -244,13 +252,13 @@ protected void createOperationLog() { } // register this operationLog to current thread - OperationLog.setCurrentOperationLog(operationLog); + operationManager.setCurrentOperationLog(operationLog, operationLogFile); } } protected void unregisterOperationLog() { if (isOperationLogEnabled) { - OperationLog.removeCurrentOperationLog(); + operationManager.removeCurrentOperationLog(); } } @@ -336,11 +344,11 @@ protected void validateFetchOrientation(FetchOrientation orientation, } } - protected HiveSQLException toSQLException(String prefix, CommandProcessorResponse response) { - HiveSQLException ex = new HiveSQLException(prefix + ": " + response.getErrorMessage(), - response.getSQLState(), response.getResponseCode()); - if (response.getException() != null) { - ex.initCause(response.getException()); + protected HiveSQLException toSQLException(String prefix, CommandProcessorException exception) { + HiveSQLException ex = new HiveSQLException(prefix + ": " + exception.getMessage(), + exception.getSqlState(), exception.getResponseCode()); + if (exception.getCause() != null) { + ex.initCause(exception.getCause()); } return ex; } diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java index 79c12d387a9d..5488a3db7fe9 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java @@ -17,6 +17,7 @@ package org.apache.hive.service.cli.operation; +import java.io.File; import java.sql.SQLException; import java.util.ArrayList; import java.util.HashMap; @@ -56,6 +57,11 @@ public class OperationManager extends AbstractService { private final Map handleToOperation = new HashMap(); + private final ThreadLocal threadLocalOperationLog = + ThreadLocal.withInitial(() -> null); + + private final ThreadLocal threadLocalFile = ThreadLocal.withInitial(() -> null); + public OperationManager() { super(OperationManager.class.getSimpleName()); } @@ -97,20 +103,20 @@ public ExecuteStatementOperation newExecuteStatementOperation(HiveSession parent } public GetTypeInfoOperation newGetTypeInfoOperation(HiveSession parentSession) { - GetTypeInfoOperation operation = new GetTypeInfoOperation(parentSession); + GetTypeInfoOperation operation = new GetTypeInfoOperation(parentSession, this); addOperation(operation); return operation; } public GetCatalogsOperation newGetCatalogsOperation(HiveSession parentSession) { - GetCatalogsOperation operation = new GetCatalogsOperation(parentSession); + GetCatalogsOperation operation = new GetCatalogsOperation(parentSession, this); addOperation(operation); return operation; } public GetSchemasOperation newGetSchemasOperation(HiveSession parentSession, String catalogName, String schemaName) { - GetSchemasOperation operation = new GetSchemasOperation(parentSession, catalogName, schemaName); + GetSchemasOperation operation = new GetSchemasOperation(parentSession, this, catalogName, schemaName); addOperation(operation); return operation; } @@ -119,20 +125,20 @@ public MetadataOperation newGetTablesOperation(HiveSession parentSession, String catalogName, String schemaName, String tableName, List tableTypes) { MetadataOperation operation = - new GetTablesOperation(parentSession, catalogName, schemaName, tableName, tableTypes); + new GetTablesOperation(parentSession, this, catalogName, schemaName, tableName, tableTypes); addOperation(operation); return operation; } public GetTableTypesOperation newGetTableTypesOperation(HiveSession parentSession) { - GetTableTypesOperation operation = new GetTableTypesOperation(parentSession); + GetTableTypesOperation operation = new GetTableTypesOperation(parentSession, this); addOperation(operation); return operation; } public GetColumnsOperation newGetColumnsOperation(HiveSession parentSession, String catalogName, String schemaName, String tableName, String columnName) { - GetColumnsOperation operation = new GetColumnsOperation(parentSession, + GetColumnsOperation operation = new GetColumnsOperation(parentSession, this, catalogName, schemaName, tableName, columnName); addOperation(operation); return operation; @@ -140,7 +146,7 @@ public GetColumnsOperation newGetColumnsOperation(HiveSession parentSession, public GetFunctionsOperation newGetFunctionsOperation(HiveSession parentSession, String catalogName, String schemaName, String functionName) { - GetFunctionsOperation operation = new GetFunctionsOperation(parentSession, + GetFunctionsOperation operation = new GetFunctionsOperation(parentSession, this, catalogName, schemaName, functionName); addOperation(operation); return operation; @@ -148,7 +154,7 @@ public GetFunctionsOperation newGetFunctionsOperation(HiveSession parentSession, public GetPrimaryKeysOperation newGetPrimaryKeysOperation(HiveSession parentSession, String catalogName, String schemaName, String tableName) { - GetPrimaryKeysOperation operation = new GetPrimaryKeysOperation(parentSession, + GetPrimaryKeysOperation operation = new GetPrimaryKeysOperation(parentSession, this, catalogName, schemaName, tableName); addOperation(operation); return operation; @@ -158,7 +164,7 @@ public GetCrossReferenceOperation newGetCrossReferenceOperation( HiveSession session, String primaryCatalog, String primarySchema, String primaryTable, String foreignCatalog, String foreignSchema, String foreignTable) { - GetCrossReferenceOperation operation = new GetCrossReferenceOperation(session, + GetCrossReferenceOperation operation = new GetCrossReferenceOperation(session, this, primaryCatalog, primarySchema, primaryTable, foreignCatalog, foreignSchema, foreignTable); addOperation(operation); @@ -283,8 +289,22 @@ private Schema getLogSchema() { return schema; } - public OperationLog getOperationLogByThread() { - return OperationLog.getCurrentOperationLog(); + public void setCurrentOperationLog(OperationLog log, File file) { + threadLocalOperationLog.set(log); + threadLocalFile.set(file); + } + + public OperationLog getCurrentOperationLog() { + return threadLocalOperationLog.get(); + } + + public File getCurrentOperationLogFile() { + return threadLocalFile.get(); + } + + public void removeCurrentOperationLog() { + threadLocalOperationLog.remove(); + threadLocalFile.remove(); } public List removeExpiredOperations(OperationHandle[] handles) { diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java index 19aa7ab385ae..764483eee3f0 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java @@ -116,8 +116,7 @@ public HiveSessionImpl(TProtocolVersion protocol, String username, String passwo try { // In non-impersonation mode, map scheduler queue to current user // if fair scheduler is configured. - if (! hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS) && - hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_MAP_FAIR_SCHEDULER_QUEUE)) { + if (!hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS)) { ShimLoader.getHadoopShims().refreshDefaultQueue(hiveConf, username); } } catch (IOException e) { @@ -273,7 +272,7 @@ private static void setConf(String varname, String key, String varvalue, boolean new VariableSubstitution(() -> SessionState.get().getHiveVariables()); HiveConf conf = SessionState.get().getConf(); String value = substitution.substitute(conf, varvalue); - if (conf.getBoolVar(HiveConf.ConfVars.HIVECONFVALIDATION)) { + if (conf.getBoolVar(HiveConf.ConfVars.HIVE_CONF_VALIDATION)) { HiveConf.ConfVars confVars = HiveConf.getConfVars(key); if (confVars != null) { if (!confVars.isType(value)) { diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java index 0ec13424fd0f..b24aefab02b0 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java @@ -23,8 +23,10 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.metadata.Hive; import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.shims.Utils; +import org.apache.hadoop.io.Text; import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hive.service.auth.HiveAuthFactory; import org.apache.hive.service.cli.HiveSQLException; import org.apache.hive.service.rpc.thrift.TProtocolVersion; @@ -96,6 +98,14 @@ public void close() throws HiveSQLException { } } + private static Token createToken(String tokenStr, String tokenService) + throws IOException { + Token delegationToken = new Token<>(); + delegationToken.decodeFromUrlString(tokenStr); + delegationToken.setService(new Text(tokenService)); + return delegationToken; + } + /** * Enable delegation token for the session * save the token string and set the token.signature in hive conf. The metastore client uses @@ -108,7 +118,8 @@ private void setDelegationToken(String delegationTokenStr) throws HiveSQLExcepti if (delegationTokenStr != null) { getHiveConf().set("hive.metastore.token.signature", HS2TOKEN); try { - Utils.setTokenStr(sessionUgi, delegationTokenStr, HS2TOKEN); + Token delegationToken = createToken(delegationTokenStr, HS2TOKEN); + sessionUgi.addToken(delegationToken); } catch (IOException e) { throw new HiveSQLException("Couldn't setup delegation token in the ugi", e); } diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java index c7fa7b5f3e0a..1cef6c8c4711 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java @@ -81,8 +81,13 @@ protected void initializeServer() { } String keyStorePassword = ShimLoader.getHadoopShims().getPassword(hiveConf, HiveConf.ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PASSWORD.varname); + String keyStoreType = hiveConf.getVar(ConfVars.HIVE_SERVER2_SSL_KEYSTORE_TYPE).trim(); + String keyStoreAlgorithm = hiveConf.getVar(ConfVars.HIVE_SERVER2_SSL_KEYMANAGERFACTORY_ALGORITHM).trim(); + String includeCiphersuites = hiveConf.getVar(ConfVars.HIVE_SERVER2_SSL_BINARY_INCLUDE_CIPHERSUITES).trim(); + serverSocket = HiveAuthUtils.getServerSSLSocket(hiveHost, portNum, keyStorePath, - keyStorePassword, sslVersionBlacklist); + keyStorePassword, keyStoreType, keyStoreAlgorithm, sslVersionBlacklist, + includeCiphersuites); } // In case HIVE_SERVER2_THRIFT_PORT or hive.server2.thrift.port is configured with 0 which diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java index 16d8540b4056..66ef2aceea8b 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java @@ -20,7 +20,7 @@ import java.util.Map; -import org.apache.hadoop.hive.metastore.HiveMetaStore; +import org.apache.hadoop.hive.metastore.HMSHandler; import org.apache.hadoop.hive.metastore.RawStore; import org.apache.spark.internal.SparkLogger; @@ -67,7 +67,7 @@ private void cleanRawStore() { */ public void cacheThreadLocalRawStore() { Long threadId = this.getId(); - RawStore threadLocalRawStore = HiveMetaStore.HMSHandler.getRawStore(); + RawStore threadLocalRawStore = HMSHandler.getRawStore(); if (threadLocalRawStore != null && !threadRawStoreMap.containsKey(threadId)) { LOG.debug("Adding RawStore: " + threadLocalRawStore + ", for the thread: " + this.getName() + " to threadRawStoreMap for future cleanup."); diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala index 350aba1ab175..534c3ee72b37 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala @@ -26,7 +26,7 @@ import scala.util.control.NonFatal import org.apache.hadoop.hive.shims.Utils import org.apache.hive.service.cli._ -import org.apache.hive.service.cli.operation.ExecuteStatementOperation +import org.apache.hive.service.cli.operation.{ExecuteStatementOperation, OperationManager} import org.apache.hive.service.cli.session.HiveSession import org.apache.hive.service.rpc.thrift.{TCLIServiceConstants, TColumnDesc, TPrimitiveTypeEntry, TRowSet, TTableSchema, TTypeDesc, TTypeEntry, TTypeId, TTypeQualifiers, TTypeQualifierValue} @@ -42,11 +42,13 @@ import org.apache.spark.util.{Utils => SparkUtils} private[hive] class SparkExecuteStatementOperation( val session: SparkSession, parentSession: HiveSession, + operationManager: OperationManager, statement: String, confOverlay: JMap[String, String], runInBackground: Boolean = true, queryTimeout: Long) - extends ExecuteStatementOperation(parentSession, statement, confOverlay, runInBackground) + extends ExecuteStatementOperation(parentSession, operationManager, statement, confOverlay, + runInBackground) with SparkOperation with Logging { diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala index a02b2ca8966d..8a3c0cf8d998 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.hive.thriftserver import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType import org.apache.hive.service.cli.OperationState -import org.apache.hive.service.cli.operation.GetCatalogsOperation +import org.apache.hive.service.cli.operation.{GetCatalogsOperation, OperationManager} import org.apache.hive.service.cli.session.HiveSession import org.apache.spark.internal.Logging @@ -34,8 +34,9 @@ import org.apache.spark.sql.SparkSession */ private[hive] class SparkGetCatalogsOperation( val session: SparkSession, - parentSession: HiveSession) - extends GetCatalogsOperation(parentSession) + parentSession: HiveSession, + operationManager: OperationManager) + extends GetCatalogsOperation(parentSession, operationManager) with SparkOperation with Logging { diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala index 6c573ceb14ec..9aeba93ce084 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala @@ -24,7 +24,7 @@ import scala.jdk.CollectionConverters._ import org.apache.hadoop.hive.ql.security.authorization.plugin.{HiveOperationType, HivePrivilegeObject} import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivilegeObjectType import org.apache.hive.service.cli._ -import org.apache.hive.service.cli.operation.GetColumnsOperation +import org.apache.hive.service.cli.operation.{GetColumnsOperation, OperationManager} import org.apache.hive.service.cli.session.HiveSession import org.apache.spark.internal.Logging @@ -46,11 +46,13 @@ import org.apache.spark.sql.types._ private[hive] class SparkGetColumnsOperation( val session: SparkSession, parentSession: HiveSession, + operationManager: OperationManager, catalogName: String, schemaName: String, tableName: String, columnName: String) - extends GetColumnsOperation(parentSession, catalogName, schemaName, tableName, columnName) + extends GetColumnsOperation(parentSession, operationManager, catalogName, schemaName, tableName, + columnName) with SparkOperation with Logging { diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala index ec0da495d874..c4a64fea0f22 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala @@ -23,7 +23,7 @@ import scala.jdk.CollectionConverters._ import org.apache.hadoop.hive.ql.security.authorization.plugin.{HiveOperationType, HivePrivilegeObjectUtils} import org.apache.hive.service.cli._ -import org.apache.hive.service.cli.operation.GetFunctionsOperation +import org.apache.hive.service.cli.operation.{GetFunctionsOperation, OperationManager} import org.apache.hive.service.cli.operation.MetadataOperation.DEFAULT_HIVE_CATALOG import org.apache.hive.service.cli.session.HiveSession @@ -42,10 +42,12 @@ import org.apache.spark.sql.SparkSession private[hive] class SparkGetFunctionsOperation( val session: SparkSession, parentSession: HiveSession, + operationManager: OperationManager, catalogName: String, schemaName: String, functionName: String) - extends GetFunctionsOperation(parentSession, catalogName, schemaName, functionName) + extends GetFunctionsOperation(parentSession, operationManager, catalogName, schemaName, + functionName) with SparkOperation with Logging { diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala index 4b8b603eede5..458b9110c209 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala @@ -21,7 +21,7 @@ import java.util.regex.Pattern import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType import org.apache.hive.service.cli._ -import org.apache.hive.service.cli.operation.GetSchemasOperation +import org.apache.hive.service.cli.operation.{GetSchemasOperation, OperationManager} import org.apache.hive.service.cli.operation.MetadataOperation.DEFAULT_HIVE_CATALOG import org.apache.hive.service.cli.session.HiveSession @@ -40,9 +40,10 @@ import org.apache.spark.sql.SparkSession private[hive] class SparkGetSchemasOperation( val session: SparkSession, parentSession: HiveSession, + operationManager: OperationManager, catalogName: String, schemaName: String) - extends GetSchemasOperation(parentSession, catalogName, schemaName) + extends GetSchemasOperation(parentSession, operationManager, catalogName, schemaName) with SparkOperation with Logging { diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala index e3e1cdbd3681..baa09a233ba2 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala @@ -21,7 +21,7 @@ import java.util.UUID import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType import org.apache.hive.service.cli._ -import org.apache.hive.service.cli.operation.GetTableTypesOperation +import org.apache.hive.service.cli.operation.{GetTableTypesOperation, OperationManager} import org.apache.hive.service.cli.session.HiveSession import org.apache.spark.internal.Logging @@ -37,8 +37,9 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTableType */ private[hive] class SparkGetTableTypesOperation( val session: SparkSession, - parentSession: HiveSession) - extends GetTableTypesOperation(parentSession) + parentSession: HiveSession, + operationManager: OperationManager) + extends GetTableTypesOperation(parentSession, operationManager) with SparkOperation with Logging { diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala index 0579d567d022..0393dd2b1824 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala @@ -24,7 +24,7 @@ import scala.jdk.CollectionConverters._ import org.apache.hadoop.hive.ql.security.authorization.plugin.{HiveOperationType, HivePrivilegeObjectUtils} import org.apache.hive.service.cli._ -import org.apache.hive.service.cli.operation.GetTablesOperation +import org.apache.hive.service.cli.operation.{GetTablesOperation, OperationManager} import org.apache.hive.service.cli.session.HiveSession import org.apache.spark.internal.Logging @@ -45,11 +45,13 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTableType._ private[hive] class SparkGetTablesOperation( val session: SparkSession, parentSession: HiveSession, + operationManager: OperationManager, catalogName: String, schemaName: String, tableName: String, tableTypes: JList[String]) - extends GetTablesOperation(parentSession, catalogName, schemaName, tableName, tableTypes) + extends GetTablesOperation(parentSession, operationManager, catalogName, schemaName, tableName, + tableTypes) with SparkOperation with Logging { diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala index 4af2265841b5..6b141bb595fa 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala @@ -23,7 +23,7 @@ import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType import org.apache.hadoop.hive.serde2.thrift.Type import org.apache.hadoop.hive.serde2.thrift.Type._ import org.apache.hive.service.cli.OperationState -import org.apache.hive.service.cli.operation.GetTypeInfoOperation +import org.apache.hive.service.cli.operation.{GetTypeInfoOperation, OperationManager} import org.apache.hive.service.cli.session.HiveSession import org.apache.spark.internal.Logging @@ -38,8 +38,9 @@ import org.apache.spark.sql.SparkSession */ private[hive] class SparkGetTypeInfoOperation( val session: SparkSession, - parentSession: HiveSession) - extends GetTypeInfoOperation(parentSession) + parentSession: HiveSession, + operationManager: OperationManager) + extends GetTypeInfoOperation(parentSession, operationManager) with SparkOperation with Logging { diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index 02826abef857..e1dd2560fa2d 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -30,6 +30,7 @@ import jline.console.history.FileHistory import org.apache.hadoop.conf.Configuration import org.apache.hadoop.hive.cli.{CliDriver, CliSessionState, OptionsProcessor} import org.apache.hadoop.hive.common.HiveInterruptUtils +import org.apache.hadoop.hive.common.io.SessionStream import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.processors._ @@ -99,9 +100,9 @@ private[hive] object SparkSQLCLIDriver extends Logging { sessionState.in = System.in try { - sessionState.out = new PrintStream(System.out, true, UTF_8.name()) - sessionState.info = new PrintStream(System.err, true, UTF_8.name()) - sessionState.err = new PrintStream(System.err, true, UTF_8.name()) + sessionState.out = new SessionStream(System.out, true, UTF_8.name()) + sessionState.info = new SessionStream(System.err, true, UTF_8.name()) + sessionState.err = new SessionStream(System.err, true, UTF_8.name()) } catch { case e: UnsupportedEncodingException => closeHiveSessionStateIfStarted(sessionState) @@ -182,9 +183,9 @@ private[hive] object SparkSQLCLIDriver extends Logging { // will set the output into an invalid buffer. sessionState.in = System.in try { - sessionState.out = new PrintStream(System.out, true, UTF_8.name()) - sessionState.info = new PrintStream(System.err, true, UTF_8.name()) - sessionState.err = new PrintStream(System.err, true, UTF_8.name()) + sessionState.out = new SessionStream(System.out, true, UTF_8.name()) + sessionState.info = new SessionStream(System.err, true, UTF_8.name()) + sessionState.err = new SessionStream(System.err, true, UTF_8.name()) } catch { case e: UnsupportedEncodingException => exit(ERROR_PATH_NOT_FOUND) } @@ -206,17 +207,27 @@ private[hive] object SparkSQLCLIDriver extends Logging { cli.printMasterAndAppId() if (sessionState.execString != null) { - exit(cli.processLine(sessionState.execString)) + try { + cli.processLine(sessionState.execString) + exit(0) + } catch { + case e: CommandProcessorException => + logError(log"Could not execute. (${MDC(ERROR, e.getMessage)})") + exit(e.getErrorCode) + } } try { if (sessionState.fileName != null) { - exit(cli.processFile(sessionState.fileName)) + cli.processFile(sessionState.fileName) } } catch { case e: FileNotFoundException => logError(log"Could not open input file for reading. (${MDC(ERROR, e.getMessage)})") exit(ERROR_PATH_NOT_FOUND) + case e: CommandProcessorException => + logError(log"Could not process input file. (${MDC(ERROR, e.getMessage)})") + exit(e.getErrorCode) } val reader = new ConsoleReader() @@ -257,7 +268,6 @@ private[hive] object SparkSQLCLIDriver extends Logging { } } - var ret = 0 var prefix = "" def currentDB = { @@ -285,7 +295,12 @@ private[hive] object SparkSQLCLIDriver extends Logging { if (line.trim().endsWith(";") && !line.trim().endsWith("\\;")) { line = prefix + line - ret = cli.processLine(line, true) + try { + cli.processLine(line, true) + } catch { + case e: CommandProcessorException => + exit(e.getErrorCode) + } prefix = "" currentPrompt = promptWithCurrentDB } else { @@ -298,7 +313,7 @@ private[hive] object SparkSQLCLIDriver extends Logging { closeHiveSessionStateIfStarted(sessionState) - exit(ret) + exit(0) } def printUsage(): Unit = { @@ -421,7 +436,8 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { console.printInfo(s"Spark master: $master, Application Id: $appId") } - override def processCmd(cmd: String): Int = { + override def processCmd(cmd: String): CommandProcessorResponse = { + val ret = new CommandProcessorResponse() val cmd_trimmed: String = cmd.trim() val cmd_lower = cmd_trimmed.toLowerCase(Locale.ROOT) val tokens: Array[String] = cmd_trimmed.split("\\s+") @@ -437,9 +453,8 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { val endTimeNs = System.nanoTime() val timeTaken: Double = TimeUnit.NANOSECONDS.toMillis(endTimeNs - startTimeNs) / 1000.0 console.printInfo(s"Time taken: $timeTaken seconds") - 0 + ret } else { - var ret = 0 val hconf = conf.asInstanceOf[HiveConf] val proc: CommandProcessor = CommandProcessorFactory.get(tokens, hconf) @@ -451,7 +466,6 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { proc.isInstanceOf[ResetProcessor] ) { val driver = new SparkSQLDriver - driver.init() val out = sessionState.out val err = sessionState.err val startTimeNs: Long = System.nanoTime() @@ -462,7 +476,6 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { driver.run(cmd) } catch { case t: Throwable => - ret = 1 val format = SparkSQLEnv.sparkSession.sessionState.conf.errorMessageFormat val msg = t match { case st: SparkThrowable with Throwable => @@ -476,7 +489,7 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { t.printStackTrace(err) } driver.close() - return ret + throw new CommandProcessorException(t) } val endTimeNs = System.nanoTime() val timeTaken: Double = TimeUnit.NANOSECONDS.toMillis(endTimeNs - startTimeNs) / 1000.0 @@ -506,13 +519,10 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { s"""Failed with exception ${e.getClass.getName}: ${e.getMessage} |${Utils.stringifyException(e)} """.stripMargin) - ret = 1 + throw new CommandProcessorException(e) } - val cret = driver.close() - if (ret == 0) { - ret = cret - } + driver.close() var responseMsg = s"Time taken: $timeTaken seconds" if (counter != 0) { @@ -525,7 +535,7 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { if (sessionState.getIsVerbose) { sessionState.out.println(tokens(0) + " " + cmd_1) } - ret = proc.run(cmd_1).getResponseCode + proc.run(cmd_1) } // scalastyle:on println } @@ -534,7 +544,7 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { } // Adapted processLine from Hive 2.3's CliDriver.processLine. - override def processLine(line: String, allowInterrupting: Boolean): Int = { + override def processLine(line: String, allowInterrupting: Boolean): CommandProcessorResponse = { var oldSignal: SignalHandler = null var interruptSignal: Signal = null @@ -566,7 +576,9 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { } try { - var lastRet: Int = 0 + val ignoreErrors = + HiveConf.getBoolVar(conf, HiveConf.getConfVars("hive.cli.errors.ignore")) + var ret: CommandProcessorResponse = null // we can not use "split" function directly as ";" may be quoted val commands = splitSemiColon(line).asScala @@ -577,20 +589,19 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { } else { command += oneCmd if (!SparkStringUtils.isBlank(command)) { - val ret = processCmd(command) - command = "" - lastRet = ret - val ignoreErrors = - HiveConf.getBoolVar(conf, HiveConf.getConfVars("hive.cli.errors.ignore")) - if (ret != 0 && !ignoreErrors) { - CommandProcessorFactory.clean(conf.asInstanceOf[HiveConf]) - return ret + try { + ret = processCmd(command) + } catch { + case e: CommandProcessorException => + if (!ignoreErrors) { + throw e + } } + command = "" } } } - CommandProcessorFactory.clean(conf.asInstanceOf[HiveConf]) - lastRet + ret } finally { // Once we are done processing the line, restore the old handler if (oldSignal != null && interruptSignal != null) { diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala index 8b9b7352fdca..a51f61213c1d 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala @@ -23,6 +23,7 @@ import scala.jdk.CollectionConverters._ import org.apache.hadoop.hive.metastore.api.{FieldSchema, Schema} import org.apache.hadoop.hive.ql.Driver +import org.apache.hadoop.hive.ql.QueryState import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse import org.apache.spark.SparkThrowable @@ -38,15 +39,12 @@ import org.apache.spark.util.Utils private[hive] class SparkSQLDriver(val sparkSession: SparkSession = SparkSQLEnv.sparkSession) - extends Driver + extends Driver(new QueryState.Builder().build()) with Logging { private[hive] var tableSchema: Schema = _ private[hive] var hiveResponse: Seq[String] = _ - override def init(): Unit = { - } - private def getResultSetSchema(query: QueryExecution): Schema = { val analyzed = query.analyzed logDebug(s"Result Schema: ${analyzed.output}") @@ -79,7 +77,7 @@ private[hive] class SparkSQLDriver(val sparkSession: SparkSession = SparkSQLEnv. } } tableSchema = getResultSetSchema(execution) - new CommandProcessorResponse(0) + new CommandProcessorResponse() } catch { case st: SparkThrowable => logDebug(s"Failed in [$command]", st) @@ -90,10 +88,9 @@ private[hive] class SparkSQLDriver(val sparkSession: SparkSession = SparkSQLEnv. } } - override def close(): Int = { + override def close(): Unit = { hiveResponse = null tableSchema = null - 0 } override def getResults(res: JList[_]): Boolean = { diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala index 8e12165dd6f1..5c81400289d2 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala @@ -52,7 +52,7 @@ private[thriftserver] class SparkSQLOperationManager() val conf = sparkSession.sessionState.conf val runInBackground = async && conf.getConf(HiveUtils.HIVE_THRIFT_SERVER_ASYNC) val operation = new SparkExecuteStatementOperation( - sparkSession, parentSession, statement, confOverlay, runInBackground, queryTimeout) + sparkSession, parentSession, this, statement, confOverlay, runInBackground, queryTimeout) handleToOperation.put(operation.getHandle, operation) logDebug(s"Created Operation for $statement with session=$parentSession, " + s"runInBackground=$runInBackground") @@ -64,7 +64,7 @@ private[thriftserver] class SparkSQLOperationManager() val session = sessionToContexts.get(parentSession.getSessionHandle) require(session != null, s"Session handle: ${parentSession.getSessionHandle} has not been" + " initialized or had already closed.") - val operation = new SparkGetCatalogsOperation(session, parentSession) + val operation = new SparkGetCatalogsOperation(session, parentSession, this) handleToOperation.put(operation.getHandle, operation) logDebug(s"Created GetCatalogsOperation with session=$parentSession.") operation @@ -77,7 +77,8 @@ private[thriftserver] class SparkSQLOperationManager() val session = sessionToContexts.get(parentSession.getSessionHandle) require(session != null, s"Session handle: ${parentSession.getSessionHandle} has not been" + " initialized or had already closed.") - val operation = new SparkGetSchemasOperation(session, parentSession, catalogName, schemaName) + val operation = new SparkGetSchemasOperation(session, parentSession, this, catalogName, + schemaName) handleToOperation.put(operation.getHandle, operation) logDebug(s"Created GetSchemasOperation with session=$parentSession.") operation @@ -92,7 +93,7 @@ private[thriftserver] class SparkSQLOperationManager() val session = sessionToContexts.get(parentSession.getSessionHandle) require(session != null, s"Session handle: ${parentSession.getSessionHandle} has not been" + " initialized or had already closed.") - val operation = new SparkGetTablesOperation(session, parentSession, + val operation = new SparkGetTablesOperation(session, parentSession, this, catalogName, schemaName, tableName, tableTypes) handleToOperation.put(operation.getHandle, operation) logDebug(s"Created GetTablesOperation with session=$parentSession.") @@ -108,7 +109,7 @@ private[thriftserver] class SparkSQLOperationManager() val session = sessionToContexts.get(parentSession.getSessionHandle) require(session != null, s"Session handle: ${parentSession.getSessionHandle} has not been" + " initialized or had already closed.") - val operation = new SparkGetColumnsOperation(session, parentSession, + val operation = new SparkGetColumnsOperation(session, parentSession, this, catalogName, schemaName, tableName, columnName) handleToOperation.put(operation.getHandle, operation) logDebug(s"Created GetColumnsOperation with session=$parentSession.") @@ -120,21 +121,22 @@ private[thriftserver] class SparkSQLOperationManager() val session = sessionToContexts.get(parentSession.getSessionHandle) require(session != null, s"Session handle: ${parentSession.getSessionHandle} has not been" + " initialized or had already closed.") - val operation = new SparkGetTableTypesOperation(session, parentSession) + val operation = new SparkGetTableTypesOperation(session, parentSession, this) handleToOperation.put(operation.getHandle, operation) logDebug(s"Created GetTableTypesOperation with session=$parentSession.") operation } - override def newGetFunctionsOperation( + def newGetFunctionsOperation( parentSession: HiveSession, + operationManager: OperationManager, catalogName: String, schemaName: String, functionName: String): GetFunctionsOperation = synchronized { val session = sessionToContexts.get(parentSession.getSessionHandle) require(session != null, s"Session handle: ${parentSession.getSessionHandle} has not been" + " initialized or had already closed.") - val operation = new SparkGetFunctionsOperation(session, parentSession, + val operation = new SparkGetFunctionsOperation(session, parentSession, operationManager, catalogName, schemaName, functionName) handleToOperation.put(operation.getHandle, operation) logDebug(s"Created GetFunctionsOperation with session=$parentSession.") @@ -146,7 +148,7 @@ private[thriftserver] class SparkSQLOperationManager() val session = sessionToContexts.get(parentSession.getSessionHandle) require(session != null, s"Session handle: ${parentSession.getSessionHandle} has not been" + " initialized or had already closed.") - val operation = new SparkGetTypeInfoOperation(session, parentSession) + val operation = new SparkGetTypeInfoOperation(session, parentSession, this) handleToOperation.put(operation.getHandle, operation) logDebug(s"Created GetTypeInfoOperation with session=$parentSession.") operation diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/GetCatalogsOperationMock.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/GetCatalogsOperationMock.scala index 1bc9aaf672c3..3f09e6045490 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/GetCatalogsOperationMock.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/GetCatalogsOperationMock.scala @@ -20,12 +20,12 @@ import java.nio.ByteBuffer import java.util.UUID import org.apache.hive.service.cli.OperationHandle -import org.apache.hive.service.cli.operation.GetCatalogsOperation +import org.apache.hive.service.cli.operation.{GetCatalogsOperation, OperationManager} import org.apache.hive.service.cli.session.HiveSession import org.apache.hive.service.rpc.thrift.{THandleIdentifier, TOperationHandle, TOperationType} -class GetCatalogsOperationMock(parentSession: HiveSession) - extends GetCatalogsOperation(parentSession) { +class GetCatalogsOperationMock(parentSession: HiveSession, operationManager: OperationManager) + extends GetCatalogsOperation(parentSession, operationManager) { override def runInternal(): Unit = {} diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveSessionImplSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveSessionImplSuite.scala index 6f89fbfb788b..eb0e91728fda 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveSessionImplSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveSessionImplSuite.scala @@ -67,7 +67,7 @@ class OperationManagerMock extends OperationManager { private val calledHandles: mutable.Set[OperationHandle] = new mutable.HashSet[OperationHandle]() override def newGetCatalogsOperation(parentSession: HiveSession): GetCatalogsOperation = { - val operation = new GetCatalogsOperationMock(parentSession) + val operation = new GetCatalogsOperationMock(parentSession, this) try { val m = classOf[OperationManager].getDeclaredMethod("addOperation", classOf[Operation]) m.setAccessible(true) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala index 6f0fedcb8536..590df2d28850 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala @@ -113,7 +113,7 @@ trait SharedThriftServer extends SharedSparkSession { val interceptor = new HttpBasicAuthInterceptor( username, "anonymous", - null, null, true, new util.HashMap[String, String]()) + null, null, true, new util.HashMap[String, String](), new util.HashMap[String, String]()) new THttpClient( s"http://localhost:$serverPort/cliservice", HttpClientBuilder.create.addInterceptorFirst(interceptor).build()) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperationSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperationSuite.scala index 5abf034c1dea..902d604c08d6 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperationSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperationSuite.scala @@ -24,6 +24,7 @@ import scala.concurrent.duration._ import org.apache.hadoop.hive.conf.HiveConf import org.apache.hive.service.cli.OperationState +import org.apache.hive.service.cli.operation.OperationManager import org.apache.hive.service.cli.session.{HiveSession, HiveSessionImpl} import org.apache.hive.service.rpc.thrift.{TProtocolVersion, TTypeId} import org.mockito.Mockito.{doReturn, mock, spy, when, RETURNS_DEEP_STUBS} @@ -93,7 +94,7 @@ class SparkExecuteStatementOperationSuite extends SparkFunSuite with SharedSpark doReturn(dataFrame, Nil: _*).when(spySparkSession).sql(statement) val executeStatementOperation = new MySparkExecuteStatementOperation(spySparkSession, - hiveSession, statement, signal, finalState) + hiveSession, new OperationManagerMock(), statement, signal, finalState) val run = new Thread() { override def run(): Unit = executeStatementOperation.runInternal() @@ -112,10 +113,11 @@ class SparkExecuteStatementOperationSuite extends SparkFunSuite with SharedSpark private class MySparkExecuteStatementOperation( session: SparkSession, hiveSession: HiveSession, + operationManager: OperationManager, statement: String, signal: Semaphore, finalState: OperationState) - extends SparkExecuteStatementOperation(session, hiveSession, statement, + extends SparkExecuteStatementOperation(session, hiveSession, operationManager, statement, new util.HashMap, false, 0) { override def cleanup(): Unit = { diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkThriftServerProtocolVersionsSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkThriftServerProtocolVersionsSuite.scala index 60afcf815361..8b0093edfaf6 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkThriftServerProtocolVersionsSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkThriftServerProtocolVersionsSuite.scala @@ -64,8 +64,9 @@ class SparkThriftServerProtocolVersionsSuite extends HiveThriftServer2TestBase { rs = new HiveQueryResultSet.Builder(connection) .setClient(client) - .setSessionHandle(sessHandle) - .setStmtHandle(stmtHandle).setMaxRows(Int.MaxValue).setFetchSize(Int.MaxValue) + .setStmtHandle(stmtHandle) + .setMaxRows(Int.MaxValue) + .setFetchSize(Int.MaxValue) .build() f(rs) } finally { @@ -133,7 +134,6 @@ class SparkThriftServerProtocolVersionsSuite extends HiveThriftServer2TestBase { rs = new HiveQueryResultSet.Builder(connection) .setClient(client) - .setSessionHandle(sessHandle) .setStmtHandle(getTableResp.getOperationHandle) .build() f(rs) diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 1d4c7b784c0a..17487b565a54 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -138,6 +138,10 @@ hive-llap-client ${hive.llap.scope} + + org.apache.hive + hive-udf + org.apache.avro diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveDateTimeUtils.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveDateTimeUtils.scala new file mode 100644 index 000000000000..49ea3dc61314 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveDateTimeUtils.scala @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import java.sql.Date + +import org.apache.hadoop.hive.common.`type`.{Date => HiveDate, Timestamp => HiveTimestamp} + +import org.apache.spark.sql.catalyst.util.DateTimeUtils + +object HiveDateTimeUtils { + def fromHiveTimestamp(t: HiveTimestamp): Long = { + DateTimeUtils.fromJavaTimestamp(t.toSqlTimestamp) + } + + def fromHiveDate(d: HiveDate): Int = { + DateTimeUtils.fromJavaDate(new Date(d.toEpochMilli)) + } + +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index 087f68ce3c79..eb7c40a9c504 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -342,7 +342,7 @@ private[hive] trait HiveInspectors { case _: BinaryObjectInspector => withNullSafe(o => o.asInstanceOf[Array[Byte]]) case _: DateObjectInspector if x.preferWritable() => - withNullSafe(o => getDateWritable(o)) + withNullSafe(o => getDaysWritable(o)) case _: DateObjectInspector => withNullSafe(o => DateTimeUtils.toJavaDate(o.asInstanceOf[Int])) case _: TimestampObjectInspector if x.preferWritable() => @@ -480,7 +480,7 @@ private[hive] trait HiveInspectors { _ => constant case poi: WritableConstantTimestampObjectInspector => val t = poi.getWritableConstantValue - val constant = DateTimeUtils.fromJavaTimestamp(t.getTimestamp) + val constant = HiveDateTimeUtils.fromHiveTimestamp(t.getTimestamp) _ => constant case poi: WritableConstantIntObjectInspector => val constant = poi.getWritableConstantValue.get() @@ -509,7 +509,7 @@ private[hive] trait HiveInspectors { System.arraycopy(writable.getBytes, 0, constant, 0, constant.length) _ => constant case poi: WritableConstantDateObjectInspector => - val constant = DateTimeUtils.fromJavaDate(poi.getWritableConstantValue.get()) + val constant = HiveDateTimeUtils.fromHiveDate(poi.getWritableConstantValue.get()) _ => constant case mi: StandardConstantMapObjectInspector => val keyUnwrapper = unwrapperFor(mi.getMapKeyObjectInspector) @@ -641,7 +641,7 @@ private[hive] trait HiveInspectors { case x: DateObjectInspector => data: Any => { if (data != null) { - DateTimeUtils.fromJavaDate(x.getPrimitiveJavaObject(data)) + HiveDateTimeUtils.fromHiveDate(x.getPrimitiveJavaObject(data)) } else { null } @@ -649,7 +649,7 @@ private[hive] trait HiveInspectors { case x: TimestampObjectInspector if x.preferWritable() => data: Any => { if (data != null) { - DateTimeUtils.fromJavaTimestamp(x.getPrimitiveWritableObject(data).getTimestamp) + HiveDateTimeUtils.fromHiveTimestamp(x.getPrimitiveWritableObject(data).getTimestamp) } else { null } @@ -657,7 +657,7 @@ private[hive] trait HiveInspectors { case ti: TimestampObjectInspector => data: Any => { if (data != null) { - DateTimeUtils.fromJavaTimestamp(ti.getPrimitiveJavaObject(data)) + HiveDateTimeUtils.fromHiveTimestamp(ti.getPrimitiveJavaObject(data)) } else { null } @@ -1029,7 +1029,7 @@ private[hive] trait HiveInspectors { private def getDateWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( - TypeInfoFactory.dateTypeInfo, getDateWritable(value)) + TypeInfoFactory.dateTypeInfo, getDaysWritable(value)) private def getTimestampWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( @@ -1094,7 +1094,7 @@ private[hive] trait HiveInspectors { new hadoopIo.BytesWritable(value.asInstanceOf[Array[Byte]]) } - private def getDateWritable(value: Any): DaysWritable = + private def getDaysWritable(value: Any): DaysWritable = if (value == null) { null } else { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala index a36501d1eaf4..869afb636d11 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala @@ -29,7 +29,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.common.FileUtils import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.conf.HiveConf.ConfVars +import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.util.VersionInfo import org.apache.hive.common.util.HiveVersionInfo @@ -62,7 +62,7 @@ private[spark] object HiveUtils extends Logging { " Note that, this a read-only conf and only used to report the built-in hive version." + " If you want a different metastore client for Spark to call, please refer to" + " spark.sql.hive.metastore.version.") - .version("1.1.1") + .version("4.1.0") .stringConf .checkValue(_ == builtinHiveVersion, "The builtin Hive version is read-only, please use spark.sql.hive.metastore.version") @@ -77,7 +77,7 @@ private[spark] object HiveUtils extends Logging { "2.0.0 through 2.3.10, " + "3.0.0 through 3.1.3 and " + "4.0.0 through 4.1.0.") - .version("1.4.0") + .version("4.1.0") .stringConf .checkValue(isCompatibleHiveVersion, "Unsupported Hive Metastore version") .createWithDefault(builtinHiveVersion) @@ -474,14 +474,14 @@ private[spark] object HiveUtils extends Logging { // You can search hive.metastore.uris in the code of HiveConf (in Hive's repo). // Then, you will find that the local metastore mode is only set to true when // hive.metastore.uris is not set. - propMap.put("hive.metastore.uris", "") + propMap.put(ConfVars.THRIFT_URIS.getVarname, "") // The execution client will generate garbage events, therefore the listeners that are generated // for the execution clients are useless. In order to not output garbage, we don't generate // these listeners. - propMap.put(ConfVars.METASTORE_PRE_EVENT_LISTENERS.varname, "") - propMap.put(ConfVars.METASTORE_EVENT_LISTENERS.varname, "") - propMap.put(ConfVars.METASTORE_END_FUNCTION_LISTENERS.varname, "") + propMap.put(ConfVars.PRE_EVENT_LISTENERS.getVarname, "") + propMap.put(ConfVars.EVENT_LISTENERS.getVarname, "") + propMap.put(ConfVars.END_FUNCTION_LISTENERS.getVarname, "") // SPARK-21451: Spark will gather all `spark.hadoop.*` properties from a `SparkConf` to a // Hadoop Configuration internally, as long as it happens after SparkContext initialized. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index d6f62284d2e7..2625fc8566d7 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -43,11 +43,10 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.{InternalRow, SQLConfHelper} import org.apache.spark.sql.catalyst.analysis.CastSupport import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.unsafe.types.UTF8String -import org.apache.spark.util.{SerializableConfiguration, Utils} import org.apache.spark.util.ArrayImplicits._ +import org.apache.spark.util.SerializableConfiguration /** * A trait for subclasses that handle table scans. @@ -96,7 +95,7 @@ class HadoopTableReader( override def makeRDDForTable(hiveTable: HiveTable): RDD[InternalRow] = makeRDDForTable( hiveTable, - Utils.classForName[Deserializer](tableDesc.getSerdeClassName), + tableDesc.getDeserializer(_broadcastedHadoopConf.value.value), filterOpt = None) /** @@ -110,7 +109,7 @@ class HadoopTableReader( */ def makeRDDForTable( hiveTable: HiveTable, - deserializerClass: Class[_ <: Deserializer], + deserializer: Deserializer, filterOpt: Option[PathFilter]): RDD[InternalRow] = { assert(!hiveTable.isPartitioned, @@ -132,11 +131,6 @@ class HadoopTableReader( val mutableRow = new SpecificInternalRow(attributes.map(_.dataType)) val deserializedHadoopRDD = hadoopRDD.mapPartitions { iter => - val hconf = broadcastedHadoopConf.value.value - val deserializer = deserializerClass.getConstructor().newInstance() - DeserializerLock.synchronized { - deserializer.initialize(hconf, localTableDesc.getProperties) - } HadoopTableReader.fillObject(iter, deserializer, attrsWithIndex, mutableRow, deserializer) } @@ -144,8 +138,7 @@ class HadoopTableReader( } override def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[InternalRow] = { - val partitionToDeserializer = partitions.map(part => - (part, part.getDeserializer.getClass.asInstanceOf[Class[Deserializer]])).toMap + val partitionToDeserializer = partitions.map(part => (part, part.getDeserializer)).toMap makeRDDForPartitionedTable(partitionToDeserializer, filterOpt = None) } @@ -160,7 +153,7 @@ class HadoopTableReader( * subdirectory of each partition being read. If None, then all files are accepted. */ def makeRDDForPartitionedTable( - partitionToDeserializer: Map[HivePartition, Class[_ <: Deserializer]], + partitionToDeserializer: Map[HivePartition, Deserializer], filterOpt: Option[PathFilter]): RDD[InternalRow] = { val hivePartitionRDDs = partitionToDeserializer.map { case (partition, partDeserializer) => val partDesc = Utilities.getPartitionDescFromTableDesc(tableDesc, partition, true) @@ -210,7 +203,7 @@ class HadoopTableReader( createHadoopRDD(partDesc, inputPathStr).mapPartitions { iter => val hconf = broadcastedHiveConf.value.value - val deserializer = localDeserializer.getConstructor().newInstance() + val deserializer = localDeserializer // SPARK-13709: For SerDes like AvroSerDe, some essential information (e.g. Avro schema // information) may be defined in table properties. Here we should merge table properties // and partition properties before initializing the deserializer. Note that partition @@ -225,14 +218,8 @@ class HadoopTableReader( }.foreach { case (key, value) => props.setProperty(key, value) } - DeserializerLock.synchronized { - deserializer.initialize(hconf, props) - } // get the table deserializer - val tableSerDe = localTableDesc.getDeserializerClass.getConstructor().newInstance() - DeserializerLock.synchronized { - tableSerDe.initialize(hconf, tableProperties) - } + val tableSerDe = localTableDesc.getDeserializer(hconf) // fill the non partition key attributes HadoopTableReader.fillObject(iter, deserializer, nonPartitionKeyAttrs, @@ -488,10 +475,11 @@ private[hive] object HadoopTableReader extends HiveInspectors with Logging { row.update(ordinal, HiveShim.toCatalystDecimal(oi, value)) case oi: TimestampObjectInspector => (value: Any, row: InternalRow, ordinal: Int) => - row.setLong(ordinal, DateTimeUtils.fromJavaTimestamp(oi.getPrimitiveJavaObject(value))) + row.setLong(ordinal, + HiveDateTimeUtils.fromHiveTimestamp(oi.getPrimitiveJavaObject(value))) case oi: DateObjectInspector => (value: Any, row: InternalRow, ordinal: Int) => - row.setInt(ordinal, DateTimeUtils.fromJavaDate(oi.getPrimitiveJavaObject(value))) + row.setInt(ordinal, HiveDateTimeUtils.fromHiveDate(oi.getPrimitiveJavaObject(value))) case oi: BinaryObjectInspector => (value: Any, row: InternalRow, ordinal: Int) => row.update(ordinal, oi.getPrimitiveJavaObject(value)) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index f1fe95b8a318..3c2f1a7c1494 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -37,9 +37,8 @@ import org.apache.hadoop.hive.metastore.{HiveMetaStoreClient, IMetaStoreClient, import org.apache.hadoop.hive.metastore.api.{Database => HiveDatabase, Table => MetaStoreApiTable, _} import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.metadata.{Hive, HiveException, Partition => HivePartition, Table => HiveTable} -import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer.HIVE_COLUMN_ORDER_ASC -import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.ql.session.SessionState +import org.apache.hadoop.hive.ql.util.DirectionUtils import org.apache.hadoop.hive.serde.serdeConstants import org.apache.hadoop.hive.serde2.MetadataTypedColumnsetSerDe import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe @@ -473,7 +472,7 @@ private[hive] class HiveClientImpl( // are sorted in ascending order, only then propagate the sortedness information // to downstream processing / optimizations in Spark // TODO: In future we can have Spark support columns sorted in descending order - val allAscendingSorted = sortColumnOrders.forall(_.getOrder == HIVE_COLUMN_ORDER_ASC) + val allAscendingSorted = sortColumnOrders.forall(_.getOrder == DirectionUtils.ASCENDING_CODE) val sortColumnNames = if (allAscendingSorted) { sortColumnOrders.map(_.getCol) @@ -875,20 +874,6 @@ private[hive] class HiveClientImpl( // Since HIVE-18238(Hive 3.0.0), the Driver.close function's return type changed // and the CommandProcessorFactory.clean function removed. driver.getClass.getMethod("close").invoke(driver) - if (version != hive.v3_0 && version != hive.v3_1 && version != hive.v4_0) { - CommandProcessorFactory.clean(conf) - } - } - - def getResponseCode(response: CommandProcessorResponse): Int = { - if (version < hive.v4_0) { - response.getResponseCode - } else { - // Since Hive 4.0, response code is removed from CommandProcessorResponse. - // Here we simply return 0 for the positive cases as for error cases it will - // throw exceptions early. - 0 - } } // Hive query needs to start SessionState. @@ -904,15 +889,9 @@ private[hive] class HiveClientImpl( proc match { case driver: Driver => try { - val response: CommandProcessorResponse = driver.run(cmd) - if (getResponseCode(response) != 0) { - // Throw an exception if there is an error in query processing. - // This works for hive 3.x and earlier versions. - throw new QueryExecutionException(response.getErrorMessage) - } + driver.run(cmd) driver.setMaxRows(maxRows) - val results = shim.getDriverResults(driver) - results + shim.getDriverResults(driver) } catch { case e @ (_: QueryExecutionException | _: SparkThrowable) => throw e @@ -932,15 +911,8 @@ private[hive] class HiveClientImpl( out.asInstanceOf[PrintStream].println(tokens(0) + " " + cmd_1) // scalastyle:on println } - val response: CommandProcessorResponse = proc.run(cmd_1) - val responseCode = getResponseCode(response) - if (responseCode != 0) { - // Throw an exception if there is an error in query processing. - // This works for hive 3.x and earlier versions. For 4.x and later versions, - // It will go to the catch block directly. - throw new QueryExecutionException(response.getErrorMessage) - } - Seq(responseCode.toString) + proc.run(cmd_1) + Seq.empty } } catch { case e: Exception => @@ -1063,12 +1035,7 @@ private[hive] class HiveClientImpl( val t = table.getTableName logDebug(s"Deleting table $t") try { - shim.getIndexes(client, "default", t, 255).foreach { index => - shim.dropIndex(client, "default", t, index.getIndexName) - } - if (!table.isIndexTable) { - shim.dropTable(client, "default", t) - } + shim.dropTable(client, "default", t) } catch { case _: NoSuchMethodError => // HIVE-18448 Hive 3.0 remove index APIs @@ -1205,7 +1172,7 @@ private[hive] object HiveClientImpl extends Logging { if (bucketSpec.sortColumnNames.nonEmpty) { hiveTable.setSortCols( bucketSpec.sortColumnNames - .map(col => new Order(col, HIVE_COLUMN_ORDER_ASC)) + .map(col => new Order(col, DirectionUtils.ASCENDING_CODE)) .toList .asJava ) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala index ef27669f5ba0..3de8105e909f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala @@ -29,11 +29,11 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.{IMetaStoreClient, PartitionDropOptions, TableType} -import org.apache.hadoop.hive.metastore.api.{Database, EnvironmentContext, Function => HiveFunction, FunctionType, Index, MetaException, PrincipalType, ResourceType, ResourceUri} +import org.apache.hadoop.hive.metastore.api.{Database, EnvironmentContext, Function => HiveFunction, FunctionType, MetaException, PrincipalType, ResourceType, ResourceUri} import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.io.AcidUtils import org.apache.hadoop.hive.ql.metadata.{Hive, HiveException, Partition, Table} -import org.apache.hadoop.hive.ql.plan.{AddPartitionDesc, DynamicPartitionCtx} +import org.apache.hadoop.hive.ql.plan.DynamicPartitionCtx import org.apache.hadoop.hive.ql.processors.{CommandProcessor, CommandProcessorFactory} import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.serde.serdeConstants @@ -211,8 +211,6 @@ private[client] sealed abstract class Shim { def listFunctions(hive: Hive, db: String, pattern: String): Seq[String] - def dropIndex(hive: Hive, dbName: String, tableName: String, indexName: String): Unit - def dropTable( hive: Hive, dbName: String, @@ -235,8 +233,6 @@ private[client] sealed abstract class Shim { def getMSC(hive: Hive): IMetaStoreClient - def getIndexes(hive: Hive, dbName: String, tableName: String, max: Short): Seq[Index] - protected def findMethod(klass: Class[_], name: String, args: Class[_]*): Method = { klass.getMethod(name, args: _*) } @@ -324,16 +320,10 @@ private[client] class Shim_v2_0 extends Shim with Logging { table: Table, parts: Seq[CatalogTablePartition], ignoreIfExists: Boolean): Unit = { - val addPartitionDesc = new AddPartitionDesc(table.getDbName, table.getTableName, ignoreIfExists) - parts.zipWithIndex.foreach { case (s, i) => - addPartitionDesc.addPartition( - s.spec.asJava, s.storage.locationUri.map(CatalogUtils.URIToString).orNull) - if (s.parameters.nonEmpty) { - addPartitionDesc.getPartition(i).setPartParams(s.parameters.asJava) - } + parts.zipWithIndex.foreach { case (s, _) => + hive.createPartition(table, s.parameters.asJava) + recordHiveCall() } - recordHiveCall() - hive.createPartitions(addPartitionDesc) } override def getAllPartitions(hive: Hive, table: Table): Seq[Partition] = { @@ -516,11 +506,6 @@ private[client] class Shim_v2_0 extends Shim with Logging { numDP: JInteger, listBucketingEnabled: JBoolean, isAcid, txnIdInLoadDynamicPartitions) } - override def dropIndex(hive: Hive, dbName: String, tableName: String, indexName: String): Unit = { - recordHiveCall() - hive.dropIndex(dbName, tableName, indexName, throwExceptionInDropIndex, deleteDataInDropIndex) - } - override def dropTable( hive: Hive, dbName: String, @@ -995,17 +980,9 @@ private[client] class Shim_v2_0 extends Shim with Logging { oldPartSpec: JMap[String, String], newPart: Partition): Unit = { recordHiveCall() - hive.renamePartition(table, oldPartSpec, newPart) + hive.renamePartition(table, oldPartSpec, newPart, 0) } - override def getIndexes( - hive: Hive, - dbName: String, - tableName: String, - max: Short): Seq[Index] = { - recordHiveCall() - hive.getIndexes(dbName, tableName, max).asScala.toSeq - } } private[client] class Shim_v2_1 extends Shim_v2_0 { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala index 33bf6ae1554c..95e9aa9cf01f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala @@ -23,7 +23,6 @@ import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.hive.ql.exec.Utilities import org.apache.hadoop.hive.ql.io.{HiveFileFormatUtils, HiveOutputFormat} import org.apache.hadoop.hive.ql.plan.FileSinkDesc -import org.apache.hadoop.hive.serde2.Serializer import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspectorUtils, StructObjectInspector} import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils @@ -89,7 +88,7 @@ case class HiveFileFormat(fileSinkConf: FileSinkDesc) // Add table properties from storage handler to hadoopConf, so any custom storage // handler settings can be set to hadoopConf HiveTableUtil.configureJobPropertiesForStorageHandler(tableDesc, conf, false) - Utilities.copyTableJobPropertiesToConf(tableDesc, conf) + Utilities.copyTableJobPropertiesToConf(tableDesc, conf.asInstanceOf[JobConf]) // Avoid referencing the outer object. val fileSinkConfSer = fileSinkConf @@ -135,12 +134,7 @@ class HiveOutputWriter( private def tableDesc = fileSinkConf.getTableInfo - private val serializer = { - val serializer = tableDesc.getDeserializerClass.getConstructor(). - newInstance().asInstanceOf[Serializer] - serializer.initialize(jobConf, tableDesc.getProperties) - serializer - } + private val serializer = tableDesc.getDeserializer(jobConf) private val hiveWriter = HiveFileFormatUtils.getHiveRecordWriter( jobConf, diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationExec.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationExec.scala index de2d15415837..4535e723c932 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationExec.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationExec.scala @@ -284,7 +284,7 @@ object HiveScriptIOSchema extends HiveInspectors { val properties = new Properties() properties.putAll(propsMap.asJava) - serde.initialize(null, properties) + serde.initialize(null, properties, null) serde } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala index 335d552fd50b..0a6776c7205e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala @@ -123,8 +123,7 @@ case class HiveTableScanExec( HiveShim.appendReadColumns(hiveConf, neededColumnIDs, neededColumnNames) - val deserializer = tableDesc.getDeserializerClass.getConstructor().newInstance() - deserializer.initialize(hiveConf, tableDesc.getProperties) + val deserializer = tableDesc.getDeserializer(hiveConf) // Specifies types and object inspectors of columns to be scanned. val structOI = ObjectInspectorUtils diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTempPath.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTempPath.scala index 3864d1601504..b4e94be7f597 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTempPath.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTempPath.scala @@ -149,7 +149,7 @@ class HiveTempPath(session: SparkSession, val hadoopConf: Configuration, path: P try { stagingDirForCreating.foreach { stagingDir => val fs: FileSystem = stagingDir.getFileSystem(hadoopConf) - if (!FileUtils.mkdir(fs, stagingDir, true, hadoopConf)) { + if (!FileUtils.mkdir(fs, stagingDir, hadoopConf)) { throw SparkException.internalError( "Cannot create staging directory '" + stagingDir.toString + "'") } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index b5d3fb699d62..06b32f9810b4 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -146,9 +146,9 @@ case class InsertIntoHiveTable( if (numDynamicPartitions > 0) { if (overwrite && table.tableType == CatalogTableType.EXTERNAL) { val numWrittenParts = writtenParts.size - val maxDynamicPartitionsKey = HiveConf.ConfVars.DYNAMICPARTITIONMAXPARTS.varname + val maxDynamicPartitionsKey = HiveConf.ConfVars.DYNAMIC_PARTITION_MAX_PARTS.varname val maxDynamicPartitions = hadoopConf.getInt(maxDynamicPartitionsKey, - HiveConf.ConfVars.DYNAMICPARTITIONMAXPARTS.defaultIntVal) + HiveConf.ConfVars.DYNAMIC_PARTITION_MAX_PARTS.defaultIntVal) if (numWrittenParts > maxDynamicPartitions) { throw QueryExecutionErrors.writePartitionExceedConfigSizeWhenDynamicPartitionError( numWrittenParts, maxDynamicPartitions, maxDynamicPartitionsKey) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala index 7f58f69269dd..4b8a460ac3f7 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala @@ -233,7 +233,7 @@ private[orc] class OrcSerializer(dataSchema: StructType, conf: Configuration) table.setProperty("columns.types", dataSchema.map(_.dataType.catalogString).mkString(":")) val serde = new OrcSerde - serde.initialize(conf, table) + serde.initialize(conf, table, null) serde } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index d58dec6f1126..0b2e03679b7a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -23,7 +23,7 @@ import java.sql.Timestamp import scala.util.Try -import org.apache.hadoop.hive.conf.HiveConf.ConfVars +import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkFiles, TestUtils} @@ -1212,7 +1212,7 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd .zip(parts) .map { case (k, v) => if (v == "NULL") { - s"$k=${ConfVars.DEFAULTPARTITIONNAME.defaultStrVal}" + s"$k=${ConfVars.DEFAULTPARTITIONNAME.getDefaultVal}" } else { s"$k=$v" } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala index b7c466f3ed6a..343396fa45d0 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala @@ -18,11 +18,10 @@ package org.apache.spark.sql.hive.execution import java.io.{DataInput, DataOutput, File, PrintWriter} -import java.util.{ArrayList, Arrays, Properties} +import java.util.{ArrayList, Arrays} import scala.jdk.CollectionConverters._ -import org.apache.hadoop.conf.Configuration import org.apache.hadoop.hive.ql.exec.UDF import org.apache.hadoop.hive.ql.metadata.HiveException import org.apache.hadoop.hive.ql.udf.{UDAFPercentile, UDFType} @@ -856,8 +855,6 @@ class TestPair(x: Int, y: Int) extends Writable with Serializable { } class PairSerDe extends AbstractSerDe { - override def initialize(p1: Configuration, p2: Properties): Unit = {} - override def getObjectInspector: ObjectInspector = { ObjectInspectorFactory .getStandardStructObjectInspector( From fe5563ec897743a41ad3352ff1cd531e328986d1 Mon Sep 17 00:00:00 2001 From: Vlad Rozov Date: Sun, 9 Mar 2025 20:05:10 +0000 Subject: [PATCH 02/19] [WIP][SPARK-51348][BUILD][SQL] Add missing dependency on org.apache.calcite:calcite-core --- pom.xml | 14 +++++++++----- sql/hive/pom.xml | 5 +++++ 2 files changed, 14 insertions(+), 5 deletions(-) diff --git a/pom.xml b/pom.xml index af15e013a139..bea4d1a77eed 100644 --- a/pom.xml +++ b/pom.xml @@ -1867,11 +1867,6 @@ org.apache.avro avro-mapred - - - org.apache.calcite - calcite-core - org.apache.calcite calcite-avatica @@ -1971,6 +1966,10 @@ org.apache.hadoop hadoop-yarn-registry + + com.jayway.jsonpath + json-path + @@ -2326,6 +2325,11 @@ hive-udf ${hive.version} + + org.apache.calcite + calcite-core + 1.38.0 + org.apache.orc diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 17487b565a54..e0d7cd0c3911 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -142,6 +142,11 @@ org.apache.hive hive-udf + + org.apache.calcite + calcite-core + + org.apache.avro From 8b35025751da3221d2e7d8d357abe64571e3417b Mon Sep 17 00:00:00 2001 From: Vlad Rozov Date: Thu, 17 Apr 2025 16:17:15 +0000 Subject: [PATCH 03/19] [WIP][SPARK-51348][BUILD][SQL] More changes --- assembly/pom.xml | 5 +++ dev/test-dependencies.sh | 4 +- pom.xml | 39 ++++++++++++++--- .../apache/spark/repl/SparkShellSuite.scala | 37 ++++++++-------- sql/hive/pom.xml | 25 ++++++++++- .../apache/spark/sql/hive/TableReader.scala | 28 +++++++++---- .../sql/hive/client/HiveClientImpl.scala | 42 ++++++------------- .../spark/sql/hive/client/HiveShim.scala | 6 +-- .../sql/hive/client/HiveClientVersions.scala | 2 +- .../sql/hive/client/HiveVersionSuite.scala | 1 - 10 files changed, 119 insertions(+), 70 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 3a6b73b4522d..f5e9bc20f37c 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -255,6 +255,11 @@ spark-hive_${scala.binary.version} ${project.version} + + org.eclipse.jetty + jetty-util + runtime + diff --git a/dev/test-dependencies.sh b/dev/test-dependencies.sh index 68c61232ea2a..03c4f7fad2eb 100755 --- a/dev/test-dependencies.sh +++ b/dev/test-dependencies.sh @@ -34,7 +34,7 @@ HADOOP_MODULE_PROFILES="-Phive-thriftserver -Pkubernetes -Pyarn -Phive \ -Pspark-ganglia-lgpl -Pkinesis-asl -Phadoop-cloud -Pjvm-profiler" MVN="build/mvn" HADOOP_HIVE_PROFILES=( - hadoop-3-hive-2.3 + hadoop-3-hive-4.0 ) MVN_EXEC_PLUGIN_VERSION=$(build/mvn help:evaluate \ @@ -85,7 +85,7 @@ $MVN -q versions:set -DnewVersion=$TEMP_VERSION -DgenerateBackupPoms=false > /de # Generate manifests for each Hadoop profile: for HADOOP_HIVE_PROFILE in "${HADOOP_HIVE_PROFILES[@]}"; do - if [[ $HADOOP_HIVE_PROFILE == **hadoop-3-hive-2.3** ]]; then + if [[ $HADOOP_HIVE_PROFILE == **hadoop-3-hive-4.0** ]]; then HADOOP_PROFILE=hadoop-3 fi echo "Performing Maven install for $HADOOP_HIVE_PROFILE" diff --git a/pom.xml b/pom.xml index bea4d1a77eed..6f84b49598ef 100644 --- a/pom.xml +++ b/pom.xml @@ -132,7 +132,7 @@ 3.9.3 5.9.0 org.apache.hive - core + 4.0.1 @@ -196,7 +196,7 @@ 3.18.0 2.12.1 - 4.1.17 + 5.2.10 33.4.0-jre 2.11.0 3.1.9 @@ -205,7 +205,7 @@ 3.0.0 2.2.11 0.16.0 - 4.13.1 + 4.9.3 1.1 4.21.0 4.21.0 @@ -278,7 +278,7 @@ compile compile compile - 2.8.1 + ${hive.version} compile compile test @@ -830,6 +830,11 @@ log4j-slf4j2-impl ${log4j.version} + + org.apache.logging.log4j + log4j-slf4j-impl + ${log4j.version} + org.apache.logging.log4j log4j-api @@ -1679,6 +1684,10 @@ commons-logging commons-logging + + org.apache.logging.log4j + log4j-slf4j-impl + @@ -2321,14 +2330,32 @@ - org.apache.hive + ${hive.group} hive-udf ${hive.version} + + ${hive.group} + hive-standalone-metastore-server + ${hive.version} + + + org.antlr + antlr-runtime + + + + org.apache.calcite calcite-core 1.38.0 + runtime @@ -2808,7 +2835,7 @@ -Xss128m -Xms4g - -Xmx4g + -Xmx8g -XX:MaxMetaspaceSize=2g -XX:ReservedCodeCacheSize=${CodeCacheSize} --enable-native-access=ALL-UNNAMED diff --git a/repl/src/test/scala/org/apache/spark/repl/SparkShellSuite.scala b/repl/src/test/scala/org/apache/spark/repl/SparkShellSuite.scala index 067f08cb6752..9f3bc4a94d1e 100644 --- a/repl/src/test/scala/org/apache/spark/repl/SparkShellSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/SparkShellSuite.scala @@ -95,6 +95,25 @@ class SparkShellSuite extends SparkFunSuite { } } + def handleException(cause: Throwable): Unit = lock.synchronized { + val message = + s""" + |======================= + |SparkShellSuite failure output + |======================= + |Spark Shell command line: ${command.mkString(" ")} + |Exception: $cause + |Failed to capture next expected output "${expectedAnswers(next)}" within $timeout. + | + |${buffer.mkString("\n")} + |=========================== + |End SparkShellSuite failure output + |=========================== + """.stripMargin + logError(message, cause) + fail(message, cause) + } + val process = new ProcessBuilder(command: _*).start() val stdinWriter = new OutputStreamWriter(process.getOutputStream, StandardCharsets.UTF_8) @@ -119,23 +138,7 @@ class SparkShellSuite extends SparkFunSuite { } ThreadUtils.awaitResult(foundAllExpectedAnswers.future, timeoutForQuery) log.info("Found all expected output.") - } catch { case cause: Throwable => - val message = - s""" - |======================= - |SparkShellSuite failure output - |======================= - |Spark Shell command line: ${command.mkString(" ")} - |Exception: $cause - |Failed to capture next expected output "${expectedAnswers(next)}" within $timeout. - | - |${buffer.mkString("\n")} - |=========================== - |End SparkShellSuite failure output - |=========================== - """.stripMargin - logError(message, cause) - fail(message, cause) + } catch { case cause: Throwable => handleException(cause) } finally { if (!process.waitFor(1, MINUTES)) { try { diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index e0d7cd0c3911..a37fc8e6131a 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -139,12 +139,33 @@ ${hive.llap.scope} - org.apache.hive + ${hive.group} hive-udf + + ${hive.group} + hive-standalone-metastore-server + test + + + + org.apache.hadoop + hadoop-hdfs + ${hadoop.version} + test + + + org.apache.datasketches + datasketches-java + 3.1.0 @@ -197,10 +218,12 @@ org.apache.derby derby + runtime org.apache.derby derbytools + runtime org.scala-lang diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index 2625fc8566d7..0ad22937950d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -27,7 +27,7 @@ import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants._ import org.apache.hadoop.hive.ql.exec.Utilities import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition, Table => HiveTable} import org.apache.hadoop.hive.ql.plan.{PartitionDesc, TableDesc} -import org.apache.hadoop.hive.serde2.Deserializer +import org.apache.hadoop.hive.serde2.{AbstractSerDe, Deserializer} import org.apache.hadoop.hive.serde2.avro.AvroSerdeUtils.AvroTableProperties import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspectorConverters, StructObjectInspector} import org.apache.hadoop.hive.serde2.objectinspector.primitive._ @@ -95,7 +95,7 @@ class HadoopTableReader( override def makeRDDForTable(hiveTable: HiveTable): RDD[InternalRow] = makeRDDForTable( hiveTable, - tableDesc.getDeserializer(_broadcastedHadoopConf.value.value), + tableDesc.getDeserializer(_broadcastedHadoopConf.value.value).getClass, filterOpt = None) /** @@ -109,7 +109,7 @@ class HadoopTableReader( */ def makeRDDForTable( hiveTable: HiveTable, - deserializer: Deserializer, + abstractSerDeClass: Class[_ <: AbstractSerDe], filterOpt: Option[PathFilter]): RDD[InternalRow] = { assert(!hiveTable.isPartitioned, @@ -131,14 +131,20 @@ class HadoopTableReader( val mutableRow = new SpecificInternalRow(attributes.map(_.dataType)) val deserializedHadoopRDD = hadoopRDD.mapPartitions { iter => - HadoopTableReader.fillObject(iter, deserializer, attrsWithIndex, mutableRow, deserializer) + val hconf = broadcastedHadoopConf.value.value + val abstractSerDe = abstractSerDeClass.getConstructor().newInstance() + DeserializerLock.synchronized { + abstractSerDe.initialize(hconf, localTableDesc.getProperties, null) + } + HadoopTableReader.fillObject(iter, abstractSerDe, attrsWithIndex, mutableRow, abstractSerDe) } deserializedHadoopRDD } override def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[InternalRow] = { - val partitionToDeserializer = partitions.map(part => (part, part.getDeserializer)).toMap + val partitionToDeserializer = partitions.map(part => + (part, part.getDeserializer.getClass)).toMap makeRDDForPartitionedTable(partitionToDeserializer, filterOpt = None) } @@ -153,9 +159,9 @@ class HadoopTableReader( * subdirectory of each partition being read. If None, then all files are accepted. */ def makeRDDForPartitionedTable( - partitionToDeserializer: Map[HivePartition, Deserializer], + partitionToDeserializer: Map[HivePartition, Class[_ <: Deserializer]], filterOpt: Option[PathFilter]): RDD[InternalRow] = { - val hivePartitionRDDs = partitionToDeserializer.map { case (partition, partDeserializer) => + val hivePartitionRDDs = partitionToDeserializer.map { case (partition, partDeserializerClass) => val partDesc = Utilities.getPartitionDescFromTableDesc(tableDesc, partition, true) val partPath = partition.getDataLocation val inputPathStr = applyFilterIfNeeded(partPath, filterOpt) @@ -174,7 +180,7 @@ class HadoopTableReader( } val broadcastedHiveConf = _broadcastedHadoopConf - val localDeserializer = partDeserializer + val localAbstractSerDeClass = partDeserializerClass val mutableRow = new SpecificInternalRow(attributes.map(_.dataType)) // Splits all attributes into two groups, partition key attributes and those that are not. @@ -203,7 +209,8 @@ class HadoopTableReader( createHadoopRDD(partDesc, inputPathStr).mapPartitions { iter => val hconf = broadcastedHiveConf.value.value - val deserializer = localDeserializer + val deserializer = localAbstractSerDeClass.getConstructor().newInstance() + .asInstanceOf[AbstractSerDe] // SPARK-13709: For SerDes like AvroSerDe, some essential information (e.g. Avro schema // information) may be defined in table properties. Here we should merge table properties // and partition properties before initializing the deserializer. Note that partition @@ -218,6 +225,9 @@ class HadoopTableReader( }.foreach { case (key, value) => props.setProperty(key, value) } + DeserializerLock.synchronized { + deserializer.initialize(hconf, props, partProps) + } // get the table deserializer val tableSerDe = localTableDesc.getDeserializer(hconf) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index 3c2f1a7c1494..e1ad13e5dd85 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -35,7 +35,8 @@ import org.apache.hadoop.hive.common.StatsSetupConst import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.{HiveMetaStoreClient, IMetaStoreClient, RetryingMetaStoreClient, TableType => HiveTableType} import org.apache.hadoop.hive.metastore.api.{Database => HiveDatabase, Table => MetaStoreApiTable, _} -import org.apache.hadoop.hive.ql.Driver +import org.apache.hadoop.hive.metastore.conf.MetastoreConf +import org.apache.hadoop.hive.ql.IDriver import org.apache.hadoop.hive.ql.metadata.{Hive, HiveException, Partition => HivePartition, Table => HiveTable} import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.ql.util.DirectionUtils @@ -192,6 +193,9 @@ private[hive] class HiveClientImpl( /** Returns the configuration for the current session. */ def conf: HiveConf = { val hiveConf = state.getConf + // Since HIVE-17626(Hive 3.0.0), need to set hive.query.reexecution.enabled=false. + HiveConf.setBoolVar(hiveConf, HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_ENABLED, false) + // Hive changed the default of datanucleus.schema.autoCreateAll from true to false // and hive.metastore.schema.verification from false to true since Hive 2.0. // For details, see the JIRA HIVE-6113, HIVE-12463 and HIVE-1841. @@ -206,8 +210,10 @@ private[hive] class HiveClientImpl( (msConnUrl != null && msConnUrl.startsWith("jdbc:derby")) } if (isEmbeddedMetaStore) { - hiveConf.setBoolean("hive.metastore.schema.verification", false) - hiveConf.setBoolean("datanucleus.schema.autoCreateAll", true) + MetastoreConf.setBoolVar(hiveConf, MetastoreConf.ConfVars.SCHEMA_VERIFICATION, false) + MetastoreConf.setBoolVar(hiveConf, MetastoreConf.ConfVars.AUTO_CREATE_ALL, true) + // TODO: check if this can be enabled back + MetastoreConf.setBoolVar(hiveConf, MetastoreConf.ConfVars.TRY_DIRECT_SQL, false) } hiveConf } @@ -867,14 +873,8 @@ private[hive] class HiveClientImpl( * in the sequence is one row. * Since upgrading the built-in Hive to 2.3, hive-llap-client is needed when * running MapReduce jobs with `runHive`. - * Since HIVE-17626(Hive 3.0.0), need to set hive.query.reexecution.enabled=false. */ protected def runHive(cmd: String, maxRows: Int = 1000): Seq[String] = withHiveState { - def closeDriver(driver: Driver): Unit = { - // Since HIVE-18238(Hive 3.0.0), the Driver.close function's return type changed - // and the CommandProcessorFactory.clean function removed. - driver.getClass.getMethod("close").invoke(driver) - } // Hive query needs to start SessionState. SessionState.start(state) @@ -887,7 +887,7 @@ private[hive] class HiveClientImpl( val cmd_1: String = cmd_trimmed.substring(tokens(0).length()).trim() val proc = shim.getCommandProcessor(tokens(0), conf) proc match { - case driver: Driver => + case driver: IDriver => try { driver.run(cmd) driver.setMaxRows(maxRows) @@ -901,7 +901,7 @@ private[hive] class HiveClientImpl( // This works for hive 4.x and later versions. throw new QueryExecutionException(Utils.stackTraceToString(e)) } finally { - closeDriver(driver) + driver.close } case _ => @@ -1381,25 +1381,7 @@ private[hive] object HiveClientImpl extends Logging { case _ => new HiveConf(conf, classOf[HiveConf]) } - val hive = try { - Hive.getWithoutRegisterFns(hiveConf) - } catch { - // SPARK-37069: not all Hive versions have the above method (e.g., Hive 2.3.9 has it but - // 2.3.8 doesn't), therefore here we fallback when encountering the exception. - case _: NoSuchMethodError => - Hive.get(hiveConf) - } - - // Follow behavior of HIVE-26633 (4.0.0), only apply the max message size when - // `hive.thrift.client.max.message.size` is set and the value is positive - Option(hiveConf.get("hive.thrift.client.max.message.size")) - .map(HiveConf.toSizeBytes(_).toInt).filter(_ > 0) - .foreach { maxMessageSize => - logDebug(s"Trying to set metastore client thrift max message to $maxMessageSize") - configureMaxThriftMessageSize(hiveConf, hive.getMSC, maxMessageSize) - } - - hive + Hive.getWithoutRegisterFns(hiveConf) } private def getFieldValue[T](obj: Any, fieldName: String): T = { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala index 3de8105e909f..5f2e7598bdc4 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala @@ -30,7 +30,7 @@ import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.{IMetaStoreClient, PartitionDropOptions, TableType} import org.apache.hadoop.hive.metastore.api.{Database, EnvironmentContext, Function => HiveFunction, FunctionType, MetaException, PrincipalType, ResourceType, ResourceUri} -import org.apache.hadoop.hive.ql.Driver +import org.apache.hadoop.hive.ql.IDriver import org.apache.hadoop.hive.ql.io.AcidUtils import org.apache.hadoop.hive.ql.metadata.{Hive, HiveException, Partition, Table} import org.apache.hadoop.hive.ql.plan.DynamicPartitionCtx @@ -109,7 +109,7 @@ private[client] sealed abstract class Shim { def getCommandProcessor(token: String, conf: HiveConf): CommandProcessor - def getDriverResults(driver: Driver): Seq[String] + def getDriverResults(driver: IDriver): Seq[String] def getMetastoreClientConnectRetryDelayMillis(conf: HiveConf): Long @@ -446,7 +446,7 @@ private[client] class Shim_v2_0 extends Shim with Logging { override def getCommandProcessor(token: String, conf: HiveConf): CommandProcessor = CommandProcessorFactory.get(Array(token), conf) - override def getDriverResults(driver: Driver): Seq[String] = { + override def getDriverResults(driver: IDriver): Seq[String] = { val res = new JArrayList[Object]() driver.getResults(res) res.asScala.map { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientVersions.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientVersions.scala index c06e2dea40f9..c24a065bcb3c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientVersions.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientVersions.scala @@ -22,6 +22,6 @@ private[client] trait HiveClientVersions { protected val versions = if (testVersions.nonEmpty) { testVersions.get.split(",").map(_.trim).filter(_.nonEmpty).toIndexedSeq } else { - IndexedSeq("2.0", "2.1", "2.2", "2.3", "3.0", "3.1", "4.0", "4.1") + IndexedSeq("4.0", "4.1") } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveVersionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveVersionSuite.scala index 1a45f6b15096..5e2284fb70b5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveVersionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveVersionSuite.scala @@ -37,7 +37,6 @@ private[client] abstract class HiveVersionSuite(version: String) extends SparkFu // Since Hive 3.0, HIVE-19310 skipped `ensureDbInit` if `hive.in.test=false`. if (version == "3.0" || version == "3.1" || version == "4.0") { hadoopConf.set("hive.in.test", "true") - hadoopConf.set("hive.query.reexecution.enabled", "false") } HiveClientBuilder.buildClient(version, hadoopConf) } From ae7810128c5df6b5b7f9be9c117e995efe3d6cd8 Mon Sep 17 00:00:00 2001 From: Vlad Rozov Date: Tue, 6 May 2025 02:44:01 +0000 Subject: [PATCH 04/19] [WIP][SPARK-51348][BUILD][SQL] More changes --- .../org/apache/spark/util/MavenUtils.scala | 11 +++++++ pom.xml | 21 ++++++++---- project/SparkBuild.scala | 7 ++-- .../catalyst/util}/HiveDateTimeUtils.scala | 31 +++++++++++++++--- .../datasources/orc/OrcSerializer.scala | 2 +- .../datasources/orc/OrcShimUtils.scala | 10 +++--- sql/hive/pom.xml | 19 +++++++++-- .../spark/sql/hive/HiveExternalCatalog.scala | 5 ++- .../spark/sql/hive/HiveInspectors.scala | 29 ++++++++-------- .../apache/spark/sql/hive/TableReader.scala | 1 + .../sql/hive/client/HiveClientImpl.scala | 13 +++++--- .../spark/sql/hive/client/HiveShim.scala | 8 +++-- .../hive/client/IsolatedClientLoader.scala | 1 + .../spark/sql/hive/client/package.scala | 28 ++++++++-------- .../spark/sql/hive/orc/OrcFileFormat.scala | 6 ++-- .../src/test/noclasspath/hive-test-udfs.jar | Bin 35660 -> 0 bytes sql/hive/src/test/resources/TestUDTF.jar | Bin 1328 -> 0 bytes .../sql/hive/HiveExternalCatalogSuite.scala | 5 ++- .../sql/hive/client/HiveClientSuite.scala | 2 +- .../sql/hive/client/HiveClientSuites.scala | 2 +- 20 files changed, 134 insertions(+), 67 deletions(-) rename sql/{hive/src/main/scala/org/apache/spark/sql/hive => core/src/main/scala/org/apache/spark/sql/catalyst/util}/HiveDateTimeUtils.scala (54%) delete mode 100644 sql/hive/src/test/noclasspath/hive-test-udfs.jar delete mode 100644 sql/hive/src/test/resources/TestUDTF.jar diff --git a/common/utils/src/main/scala/org/apache/spark/util/MavenUtils.scala b/common/utils/src/main/scala/org/apache/spark/util/MavenUtils.scala index 5e923ad35a3a..82280adec594 100644 --- a/common/utils/src/main/scala/org/apache/spark/util/MavenUtils.scala +++ b/common/utils/src/main/scala/org/apache/spark/util/MavenUtils.scala @@ -152,6 +152,17 @@ private[spark] object MavenUtils extends Logging { cr.add(localM2) } + if (SparkEnvUtils.isTesting) { + val localM2 = new IBiblioResolver + localM2.setM2compatible(true) + localM2.setRoot( + new File(System.getProperty("user.home"), ".m2" + File.separator + "repository") + .toURI.toString) + localM2.setUsepoms(true) + localM2.setName("user-home-m2-cache") + cr.add(localM2) + } + val localIvy = new FileSystemResolver val localIvyRoot = new File(defaultIvyUserDir, "local") localIvy.setLocal(true) diff --git a/pom.xml b/pom.xml index 6f84b49598ef..a7d750e81006 100644 --- a/pom.xml +++ b/pom.xml @@ -132,9 +132,9 @@ 3.9.3 5.9.0 org.apache.hive - + core - 4.0.1 + 4.1.0-SNAPSHOT 3.9.1 @@ -1817,8 +1817,12 @@ hive-storage-api - org.apache.tez - tez-api + org.apache.hadoop + hadoop-yarn-common + + + org.apache.hadoop + hadoop-hdfs-client @@ -2345,12 +2349,12 @@ - + org.apache.calcite calcite-core @@ -2509,6 +2513,11 @@ datanucleus-core ${datanucleus-core.version} + + org.datanucleus + datanucleus-api-jdo + ${datanucleus-core.version} + org.apache.thrift libthrift diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 3ef0e174fe78..d99fa67c1212 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -1090,10 +1090,12 @@ object KubernetesIntegrationTests { */ object DependencyOverrides { lazy val guavaVersion = sys.props.get("guava.version").getOrElse("33.4.0-jre") + lazy val log4jVersion = sys.props.get("log4j.version").getOrElse("2.24.3") lazy val settings = Seq( dependencyOverrides += "com.google.guava" % "guava" % guavaVersion, dependencyOverrides += "jline" % "jline" % "2.14.6", - dependencyOverrides += "org.apache.avro" % "avro" % "1.12.0") + dependencyOverrides += "org.apache.avro" % "avro" % "1.12.0", + dependencyOverrides += "org.apache.logging.log4j" % "log4j-slf4j-impl" % log4jVersion) } /** @@ -1105,8 +1107,7 @@ object ExcludedDependencies { libraryDependencies ~= { libs => libs.filterNot(_.name == "groovy-all") }, excludeDependencies ++= Seq( ExclusionRule(organization = "ch.qos.logback"), - ExclusionRule("org.slf4j", "slf4j-simple"), - ExclusionRule("javax.servlet", "javax.servlet-api")) + ExclusionRule("org.slf4j", "slf4j-simple")) ) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveDateTimeUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/util/HiveDateTimeUtils.scala similarity index 54% rename from sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveDateTimeUtils.scala rename to sql/core/src/main/scala/org/apache/spark/sql/catalyst/util/HiveDateTimeUtils.scala index 49ea3dc61314..f409281da63e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveDateTimeUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/util/HiveDateTimeUtils.scala @@ -15,21 +15,42 @@ * limitations under the License. */ -package org.apache.spark.sql.hive +package org.apache.spark.sql.catalyst.util -import java.sql.Date +import java.sql.{Timestamp => SqlTimestamp} +import java.time.ZoneId import org.apache.hadoop.hive.common.`type`.{Date => HiveDate, Timestamp => HiveTimestamp} -import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.catalyst.util.RebaseDateTime object HiveDateTimeUtils { + private val zoneId = ZoneId.systemDefault() + + private def toSqlTimestamp(t: HiveTimestamp): SqlTimestamp = { + val ts = new SqlTimestamp(t.toEpochMilli(zoneId)) + ts.setNanos (t.getNanos) + ts + } + def fromHiveTimestamp(t: HiveTimestamp): Long = { - DateTimeUtils.fromJavaTimestamp(t.toSqlTimestamp) + DateTimeUtils.fromJavaTimestamp(toSqlTimestamp(t)) } def fromHiveDate(d: HiveDate): Int = { - DateTimeUtils.fromJavaDate(new Date(d.toEpochMilli)) + d.toEpochDay + } + + def toHiveTimestamp(t: Long): HiveTimestamp = { + val javaTimestamp = DateTimeUtils.toJavaTimestamp(t) + val hiveTimestamp = HiveTimestamp.ofEpochMilli(javaTimestamp.getTime, zoneId) + hiveTimestamp.setNanos(javaTimestamp.getNanos) + hiveTimestamp + } + + def toHiveDate(d: Int): HiveDate = { + val julian = RebaseDateTime.rebaseGregorianToJulianDays(d) + HiveDate.ofEpochDay(julian) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcSerializer.scala index 8b74405b74f7..33bad3b45c0f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcSerializer.scala @@ -137,7 +137,7 @@ class OrcSerializer(dataSchema: StructType) { new BytesWritable(getter.getBinary(ordinal)) case DateType => - OrcShimUtils.getDateWritable(reuseObj) + OrcShimUtils.getDateWritableV2(reuseObj) // The following cases are already expensive, reusing object or not doesn't matter. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala index ec33af76afbd..04c023cb978f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala @@ -24,7 +24,7 @@ import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf.{Operator => OrcOperator} import org.apache.hadoop.hive.serde2.io.{DateWritableV2, HiveDecimalWritable} import org.apache.spark.sql.catalyst.expressions.SpecializedGetters -import org.apache.spark.sql.execution.datasources.DaysWritable +import org.apache.spark.sql.catalyst.util.HiveDateTimeUtils import org.apache.spark.sql.types.Decimal /** @@ -38,7 +38,7 @@ private[sql] object OrcShimUtils { private[sql] type SearchArgument = OrcSearchArgument def getGregorianDays(value: Any): Int = { - new DaysWritable(value.asInstanceOf[DateWritableV2]).gregorianDays + HiveDateTimeUtils.fromHiveDate(value.asInstanceOf[DateWritableV2].get) } def getDecimal(value: Any): Decimal = { @@ -46,15 +46,15 @@ private[sql] object OrcShimUtils { Decimal(decimal.bigDecimalValue, decimal.precision(), decimal.scale()) } - def getDateWritable(reuseObj: Boolean): (SpecializedGetters, Int) => DateWritableV2 = { + def getDateWritableV2(reuseObj: Boolean): (SpecializedGetters, Int) => DateWritableV2 = { if (reuseObj) { - val result = new DaysWritable() + val result = new DateWritableV2() (getter, ordinal) => result.set(getter.getInt(ordinal)) result } else { (getter: SpecializedGetters, ordinal: Int) => - new DaysWritable(getter.getInt(ordinal)) + new DateWritableV2(HiveDateTimeUtils.toHiveDate(getter.getInt(ordinal))) } } diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index a37fc8e6131a..c05699d63bfc 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -147,15 +147,14 @@ hive-standalone-metastore-server test - + org.apache.hadoop hadoop-hdfs @@ -202,6 +201,10 @@ org.datanucleus datanucleus-core + + org.datanucleus + datanucleus-api-jdo + org.apache.hadoop hadoop-client-runtime @@ -215,6 +218,16 @@ org.apache.thrift libfb303 + + com.esotericsoftware.kryo + kryo5 + 5.5.0 + + + org.jodd + jodd-util + 6.0.0 + org.apache.derby derby diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index 6389e130b6a2..2568354d8f59 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -854,7 +854,10 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat // Get the original table properties as defined by the user. table.copy( createVersion = version, - properties = table.properties.filterNot { case (key, _) => key.startsWith(SPARK_SQL_PREFIX) }) + properties = table.properties.filterNot { case (key, value) => + key.startsWith(SPARK_SQL_PREFIX) || + key == "bucketing_version" && value == "2" || + key == "numFilesErasureCoded" && value == "0"}) } // Reorder table schema to put partition columns at the end. Before Spark 2.2, the partition diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index eb7c40a9c504..1e2969eb4297 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -35,7 +35,6 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.errors.DataTypeErrors.toSQLType -import org.apache.spark.sql.execution.datasources.DaysWritable import org.apache.spark.sql.types import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -93,8 +92,8 @@ import org.apache.spark.unsafe.types.UTF8String * org.apache.hadoop.hive.serde2.io.ShortWritable * org.apache.hadoop.hive.serde2.io.ByteWritable * org.apache.hadoop.io.BytesWritable - * org.apache.hadoop.hive.serde2.io.DateWritable - * org.apache.hadoop.hive.serde2.io.TimestampWritable + * org.apache.hadoop.hive.serde2.io.DateWritableV2 + * org.apache.hadoop.hive.serde2.io.TimestampWritableV2 * org.apache.hadoop.hive.serde2.io.HiveDecimalWritable * Complex Type * List: Object[] / java.util.List @@ -192,7 +191,9 @@ private[hive] trait HiveInspectors { case c: Class[_] if c == classOf[hiveIo.ByteWritable] => ByteType case c: Class[_] if c == classOf[hiveIo.ShortWritable] => ShortType case c: Class[_] if c == classOf[hiveIo.DateWritable] => DateType + case c: Class[_] if c == classOf[hiveIo.DateWritableV2] => DateType case c: Class[_] if c == classOf[hiveIo.TimestampWritable] => TimestampType + case c: Class[_] if c == classOf[hiveIo.TimestampWritableV2] => TimestampType case c: Class[_] if c == classOf[hadoopIo.Text] => StringType case c: Class[_] if c == classOf[hadoopIo.IntWritable] => IntegerType case c: Class[_] if c == classOf[hadoopIo.LongWritable] => LongType @@ -342,11 +343,11 @@ private[hive] trait HiveInspectors { case _: BinaryObjectInspector => withNullSafe(o => o.asInstanceOf[Array[Byte]]) case _: DateObjectInspector if x.preferWritable() => - withNullSafe(o => getDaysWritable(o)) + withNullSafe(o => getDateWritableV2(o)) case _: DateObjectInspector => withNullSafe(o => DateTimeUtils.toJavaDate(o.asInstanceOf[Int])) case _: TimestampObjectInspector if x.preferWritable() => - withNullSafe(o => getTimestampWritable(o)) + withNullSafe(o => getTimestampWritableV2(o)) case _: TimestampObjectInspector => withNullSafe(o => DateTimeUtils.toJavaTimestamp(o.asInstanceOf[Long])) case _: HiveIntervalDayTimeObjectInspector if x.preferWritable() => @@ -633,7 +634,7 @@ private[hive] trait HiveInspectors { case x: DateObjectInspector if x.preferWritable() => data: Any => { if (data != null) { - new DaysWritable(x.getPrimitiveWritableObject(data)).gregorianDays + HiveDateTimeUtils.fromHiveDate(x.getPrimitiveWritableObject(data).get()) } else { null } @@ -654,10 +655,10 @@ private[hive] trait HiveInspectors { null } } - case ti: TimestampObjectInspector => + case x: TimestampObjectInspector => data: Any => { if (data != null) { - HiveDateTimeUtils.fromHiveTimestamp(ti.getPrimitiveJavaObject(data)) + HiveDateTimeUtils.fromHiveTimestamp(x.getPrimitiveJavaObject(data)) } else { null } @@ -1029,11 +1030,11 @@ private[hive] trait HiveInspectors { private def getDateWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( - TypeInfoFactory.dateTypeInfo, getDaysWritable(value)) + TypeInfoFactory.dateTypeInfo, getDateWritableV2(value)) private def getTimestampWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( - TypeInfoFactory.timestampTypeInfo, getTimestampWritable(value)) + TypeInfoFactory.timestampTypeInfo, getTimestampWritableV2(value)) private def getDecimalWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( @@ -1094,18 +1095,18 @@ private[hive] trait HiveInspectors { new hadoopIo.BytesWritable(value.asInstanceOf[Array[Byte]]) } - private def getDaysWritable(value: Any): DaysWritable = + private def getDateWritableV2(value: Any): hiveIo.DateWritableV2 = if (value == null) { null } else { - new DaysWritable(value.asInstanceOf[Int]) + new hiveIo.DateWritableV2(HiveDateTimeUtils.toHiveDate(value.asInstanceOf[Int])) } - private def getTimestampWritable(value: Any): hiveIo.TimestampWritable = + private def getTimestampWritableV2(value: Any): hiveIo.TimestampWritableV2 = if (value == null) { null } else { - new hiveIo.TimestampWritable(DateTimeUtils.toJavaTimestamp(value.asInstanceOf[Long])) + new hiveIo.TimestampWritableV2(HiveDateTimeUtils.toHiveTimestamp(value.asInstanceOf[Long])) } private def getHiveIntervalDayTimeWritable(value: Any): hiveIo.HiveIntervalDayTimeWritable = diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index 0ad22937950d..3d4cfb9f3dd6 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -43,6 +43,7 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.{InternalRow, SQLConfHelper} import org.apache.spark.sql.catalyst.analysis.CastSupport import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.util.HiveDateTimeUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.ArrayImplicits._ diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index e1ad13e5dd85..43d38b29901c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -36,6 +36,7 @@ import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.{HiveMetaStoreClient, IMetaStoreClient, RetryingMetaStoreClient, TableType => HiveTableType} import org.apache.hadoop.hive.metastore.api.{Database => HiveDatabase, Table => MetaStoreApiTable, _} import org.apache.hadoop.hive.metastore.conf.MetastoreConf +import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.{AUTO_CREATE_ALL, METASTORE_METADATA_TRANSFORMER_CLASS, SCHEMA_VERIFICATION, STATS_AUTO_GATHER} import org.apache.hadoop.hive.ql.IDriver import org.apache.hadoop.hive.ql.metadata.{Hive, HiveException, Partition => HivePartition, Table => HiveTable} import org.apache.hadoop.hive.ql.session.SessionState @@ -67,6 +68,7 @@ import org.apache.spark.sql.hive.{HiveExternalCatalog, HiveUtils} import org.apache.spark.sql.hive.HiveExternalCatalog.DATASOURCE_SCHEMA import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.{CircularBuffer, Utils} /** @@ -210,10 +212,12 @@ private[hive] class HiveClientImpl( (msConnUrl != null && msConnUrl.startsWith("jdbc:derby")) } if (isEmbeddedMetaStore) { - MetastoreConf.setBoolVar(hiveConf, MetastoreConf.ConfVars.SCHEMA_VERIFICATION, false) - MetastoreConf.setBoolVar(hiveConf, MetastoreConf.ConfVars.AUTO_CREATE_ALL, true) + MetastoreConf.setBoolVar(hiveConf, SCHEMA_VERIFICATION, false) + MetastoreConf.setBoolVar(hiveConf, AUTO_CREATE_ALL, true) + MetastoreConf.setVar(hiveConf, METASTORE_METADATA_TRANSFORMER_CLASS, "") + MetastoreConf.setBoolVar(hiveConf, STATS_AUTO_GATHER, true) // TODO: check if this can be enabled back - MetastoreConf.setBoolVar(hiveConf, MetastoreConf.ConfVars.TRY_DIRECT_SQL, false) + // MetastoreConf.setBoolVar(hiveConf, MetastoreConf.ConfVars.TRY_DIRECT_SQL, false) } hiveConf } @@ -1191,8 +1195,9 @@ private[hive] object HiveClientImpl extends Logging { p: CatalogTablePartition, ht: HiveTable): HivePartition = { val tpart = new org.apache.hadoop.hive.metastore.api.Partition + val spec = new CaseInsensitiveStringMap(p.spec.asJava).asScala.view val partValues = ht.getPartCols.asScala.map { hc => - p.spec.getOrElse(hc.getName, throw new IllegalArgumentException( + spec.getOrElse(hc.getName, throw new IllegalArgumentException( s"Partition spec is missing a value for column '${hc.getName}': ${p.spec}")) } val storageDesc = new StorageDescriptor diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala index 5f2e7598bdc4..5e6a2f11e76c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala @@ -520,6 +520,7 @@ private[client] class Shim_v2_0 extends Shim with Logging { override def alterTable(hive: Hive, tableName: String, table: Table): Unit = { recordHiveCall() alterTableMethod.invoke(hive, tableName, table) + // hive.alterTable(tableName, table, ) } override def alterPartitions(hive: Hive, tableName: String, newParts: JList[Partition]): Unit = { @@ -906,6 +907,7 @@ private[client] class Shim_v2_0 extends Shim with Logging { override def createTable(hive: Hive, table: Table, ifNotExists: Boolean): Unit = { recordHiveCall() + table.setNumBuckets(0) hive.createTable(table, ifNotExists) } @@ -1263,7 +1265,7 @@ private[client] class Shim_v3_0 extends Shim_v2_3 { private[client] class Shim_v3_1 extends Shim_v3_0 -private[client] class Shim_v4_0 extends Shim_v3_1 { +private[client] class Shim_v4_1 extends Shim_v3_1 { private lazy val clazzLoadFileType = getClass.getClassLoader.loadClass( "org.apache.hadoop.hive.ql.plan.LoadTableDesc$LoadFileType") private lazy val clazzLoadTableDesc = getClass.getClassLoader.loadClass( @@ -1375,7 +1377,7 @@ private[client] class Shim_v4_0 extends Shim_v3_1 { } assert(loadFileType.isDefined) recordHiveCall() - val resetStatistics = false + val resetStatistics = true val isDirectInsert = false loadTableMethod.invoke( hive, @@ -1414,7 +1416,7 @@ private[client] class Shim_v4_0 extends Shim_v3_1 { val partitions = parts.map(HiveClientImpl.toHivePartition(_, table).getTPartition).asJava recordHiveCall() val needResults = false - addPartitionsMethod.invoke(hive, partitions, ignoreIfExists, needResults: JBoolean) + hive.addPartitions(partitions, ignoreIfExists, needResults: JBoolean) } override def loadPartition( diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala index 0bfe588df72c..8c13e2b458e8 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala @@ -210,6 +210,7 @@ private[hive] class IsolatedClientLoader( name.startsWith("org.apache.hadoop.") && !name.startsWith("org.apache.hadoop.hive.") name.startsWith("org.slf4j") || + name.startsWith("org.apache.derby.") || // derby JDBC for embedded HMS name.startsWith("org.apache.log4j") || // log4j1.x name.startsWith("org.apache.logging.log4j") || // log4j2 name.startsWith("org.apache.spark.") || diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/package.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/package.scala index d7a0c58b4016..1657083153c4 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/package.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/package.scala @@ -102,21 +102,19 @@ package object client { "org.apache.hive:hive-vector-code-gen")) case object v4_1 extends HiveVersion("4.1.0", - extraDeps = - "org.antlr:antlr4-runtime:4.9.3" :: - "org.apache.derby:derby:10.14.1.0" :: - "org.apache.hadoop:hadoop-hdfs:3.4.1" :: - "org.datanucleus:datanucleus-api-jdo:6.0.5" :: - "org.datanucleus:datanucleus-core:6.0.11" :: - "org.datanucleus:datanucleus-rdbms:6.0.10" :: - "org.datanucleus:javax.jdo:3.2.1" :: - "org.springframework:spring-core:5.3.39" :: - "org.springframework:spring-jdbc:5.3.39" :: Nil, - exclusions = - "org.apache.curator:*" :: - "org.apache.hive:hive-service-rpc" :: - "org.apache.tez:tez-api" :: - "org.apache.zookeeper:zookeeper" :: Nil) + extraDeps = Seq("org.apache.hadoop:hadoop-hdfs:3.3.6", + "org.datanucleus:datanucleus-api-jdo:5.2.10", + "org.datanucleus:datanucleus-rdbms:5.2.10", + "org.datanucleus:datanucleus-core:5.2.10", + "org.datanucleus:javax.jdo:3.2.0-release", + "org.springframework:spring-core:5.3.21", + "org.springframework:spring-jdbc:5.3.21", + "org.antlr:antlr4-runtime:4.9.3", + "org.apache.derby:derby:10.16.1.1"), + exclusions = Seq("org.apache.calcite:calcite-druid", + "org.apache.curator:*", + "org.pentaho:pentaho-aggdesigner-algorithm", + "org.apache.hive:hive-vector-code-gen")) val allSupportedHiveVersions: Set[HiveVersion] = Set(v2_0, v2_1, v2_2, v2_3, v3_0, v3_1, v4_0, v4_1) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala index 4b8a460ac3f7..b622ea79da9b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala @@ -28,7 +28,7 @@ import org.apache.commons.codec.binary.Base64 import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.hive.ql.io.orc._ -import org.apache.hadoop.hive.ql.io.sarg.SearchArgument +import org.apache.hadoop.hive.ql.io.sarg.{ConvertAstToSearchArg, SearchArgument} import org.apache.hadoop.hive.serde2.objectinspector import org.apache.hadoop.hive.serde2.objectinspector.{SettableStructObjectInspector, StructObjectInspector} import org.apache.hadoop.hive.serde2.typeinfo.{StructTypeInfo, TypeInfoUtils} @@ -137,7 +137,7 @@ case class OrcFileFormat() extends FileFormat with DataSourceRegister with Seria if (sparkSession.sessionState.conf.orcFilterPushDown) { // Sets pushed predicates OrcFilters.createFilter(requiredSchema, filters).foreach { f => - hadoopConf.set(OrcFileFormat.SARG_PUSHDOWN, toKryo(f)) + hadoopConf.set(ConvertAstToSearchArg.SARG_PUSHDOWN, ConvertAstToSearchArg.sargToKryo(f)) hadoopConf.setBoolean("hive.optimize.index.filter", true) } } @@ -318,8 +318,6 @@ private[orc] class OrcOutputWriter( } private[orc] object OrcFileFormat extends HiveInspectors with Logging { - // This constant duplicates `OrcInputFormat.SARG_PUSHDOWN`, which is unfortunately not public. - private[orc] val SARG_PUSHDOWN = "sarg.pushdown" def unwrapOrcStructs( conf: Configuration, diff --git a/sql/hive/src/test/noclasspath/hive-test-udfs.jar b/sql/hive/src/test/noclasspath/hive-test-udfs.jar deleted file mode 100644 index a5bfa456f6686da0ca010061cc3d690bfcd2ac96..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 35660 zcmb@t1yrP2vNcLkI7#8|?oQ$E?(XjHP8II%E`@91?(XjH?hXZyo}Ty5?Y{rK?s@mJ zSh+GGbD#4;?AQ_e#3w8983YRGw=bK*G?{-q`R4-&5EhV)y$P+7A|%jff@@Ii`D0&g zCs$~oPat=nfPg@dKHew$Z$AzF&rj>w>KT|C{Y?-9A0OHOx&?$kf|%+V+Su6sj>Z15 znSZmP-!=vTWc-1Zdnp)I{M%m9fPgUnuUIsK3OWjoHugsUxJsT6P=o(|m1oIq;yt`r zo$pD)ZK}}_Uy!lBU}>_~nmfVNIUyh^oXXab5H0eGipSOqkR#!Cf=|vxJ;>sAquQ|? z{R${L+nklZ03W!BeHqlESrq~L7SEv!&kG6!wHga-kb~xppD!89KB~RQthEtd$`dPsZ>DoRpOb-GeDrP?09%@Y?FL$0(cvu{Cj4o9LD z2H)DxT;;L*l8Uvuww=wv;M43F!-}VNOp{H!9!`f-Z3Qik%eFyVP7buWffLqBj?BG> zVef&OZea>NBp&))Wvs%q)Gpx&m7-}yhhN^sila4W8Et%-cMlKHZuOW0;86J|8bO6g z8EGpmFF^x^2T3aHCj3=5r99gG8qY?KcbyJi7Z}304ZL`8qlm2KQ>Tt;_;3)$(T3i- zrZ8N)B*`>6dy44DW0M%}gHm)^{)tma<+I~ z2$YbTe?Q*6>EOjnHVD?7qBKf=%_3B@P{XQdhw6NY09ucux2Z-tkbn3GNT2Y>`%tp} z6^<|7e~cv5|LcF>)Xe#xq3L$?Z^P+#b{zZ(5cl5$8rWDn+MDVB9;b4D1#J5#+7bOV zu#=&q@$cwFe??2dqCb+F>TjYOx$0TjTK@Bo)A{dL`JHROT?0Hp;4BOX5RfJ$5Rkxs zxQd98wUNDyNYn%;JgQIWFyuINriw0>^sA8@9Q}^oxuO(G~LNT z-J;NEW2E#d-bb5W(hR__K~n@3@KJl#C$6vwvGucYyJb-PqIs_5O7?Z#eY6N0oqjaQ zuWJ7i%*DmOrl*u|($;ym!gjS&C1=%syJXq97g5{BwOb9P9U{M_sa8&Nc6v1hY@D}f7o1P=;uS9B zuF0~rRG+HAG+l|jFRGKzadD)dLWxGkfEGb#EA&Ha%-O8z#|W8^Aj^UutD}&d`Gbhg zNf;#ci0wHJel-mRLoCvv?*(`$DRnoUbNUF6>#R+;Q=;H*Aj1(E6vvK2>QKe`Kmm+- zo7Q>XE$e$#V^j4b-3gfXI8FmgQm8Xrsc;#A#u>zlOkmfS`ZGFLtfd&ZH#B&De9JRU zCcQ1{a5U9=TTEi;&uE5dhvDnpaDmjTT`EjbTH?GkvA@WnjaI)_>FXg|fVJ1#!k(Zo zR}IoG8cC( z)T7-g$Y;-4t;ytTE^wLD-ebYY)_V-(yWO!KHvk@F<{u+u=gp|$dzQP(4NyWgvpjaI z>qR1+C(-AnSuI7FFC6DY)l}K%9&ws27yQ*#Rx9QCTWEL#@mA`2Vv*-UvOH(rg815i z@+&FE`0`~Ikh3WI@1XW+MA*C%`TQf&cE5E_wM|E`a7z=X!=y947WzGR^BEd4{Fgk+ z+T^SK+v0z!;4&06Y!LX6-oO&&k8*28>(tNc@2YES4Q_?}=*>cj1mzLyTDPk>0_dy4 zJn(<1J?BQi{+2$kM)_sG_xDQOYqql;q_G{0zCJ#kaeO0njTb<1aai=t?{ogbu{gJ_ zjq*_A+F5n0=|9tnbhmAQ(^T94bPc2fA2L=90 z>1-dI9-)=CmBz+3=5#o_8Hj?m6D?Vnj7Iy2j`LSs%l?2OuM-aWgh2B>sP1_=E_h@a zcrUEaXE-ZIbRCb05Sm6 zEMTW0 z%0wF-2X-Jb3%xhv9GbYZRhQ$>2x51_X9>Tf%;?dW0Wrqul(tIUtRYdofL(cD>uw^m z$j{LOO4GOIvdKgpxrazt20Gn$`tuMy~Hgg zLtqG))1eKGLtJQyX{16V;$ynqW8FTph@I5fM7vP-T3H4*-a!Axuiurv#mh%FioW^S z;v+h}KBANNKa0-)BlNC>KW}u*2COO?hf}$gmELlOq67>{A(yy@>2c~5F%s3+k)jurQp9_-Ws^%uF?lr)Dc4cH zHq#9s=Zy%i%E9*-E2Y&FHg}(i77}unw@8cJaey$tc>Hw^f>>#d_WFZL2ZvXjxQX|IY(DXOn|Yw zhxLJ_{;qeFjoMEU?SxD&zUPK^tlBFc6hHo}X4>sQ0cG0B*>D$QuFlpz2||cB(s)@w z1V+3bUPA?dh-4VGP7!fEweC!a+gH$ud-)mtusz&$HD)!bJFlt4^v5NRn_4!wSSD5*m&FL9|I zsW7@58?0ZUkt0A!RuJ=@M+tCZB4Xf{XPmLw1{YJS8Q3&q9R1cm3>jDP1vsq>a#XI& zGc^;^JT6yEo2PHM$ry7aIXlaI0SzsISn7qyOa`1ix;e=xi|*bIXmBzUAF@OZflV`e zW)@EPnqUZoM$g?aS5!1is@!ORhwy4EDP;FsHFNd^EIwf~msUTZEefBbQ>Tk+$cTMR z59ve` zwEmTh{60s$%f9isl%RRsC|Um=mwb7dgif7J8+=M!=#8VOS`eG#7k zgRp@3mq0LN0gOO9`MzMyWQNW7P&s4+ig7wJ>BLI9DuBRjiv94Lfck2FpTAP|>j+@W z&^)u7WDT@ylEEsWSF&#i@`7YyY1b^v`DQiqAd`5(sc;*$oH&q~-^4QMMPX#M!gk{h z?baec_L9V^y_Sf~(0Ta|iiGyYQ!ZJkeY4r9#*`YG@gmW4VmKuM)O!Mna+Q3gyVdDV z+_;39^w#v9_LXTbu^C-)1Qq8&Y*hD}g^g+f;JkiMrDHB#SY*75|@XX;?xHxd?3{FK2Vj;ppiqA7y8 z>~uSrgAR#JZ8~R%a-f*LVD||X{n=Rsk9>#S-A)!%De+e&Oio;kxKvsa)V`&)+-@;T zkB{h8CrQ=dLXn2QE#G4o*AI@=#ayE%9ij%pN*kh-p}`Gt20|N)XI%46`oSbY7G(VA zL(L<%fZeIsh}f+0^!_jE`9loV^ah_tiwJJkLL~vEhT;h3S>Y4&qlEYJ{;@TtKwE-IhHt+Nze3I0rt%x`eEOpT z&YUAMUVivW3m*!|_dg@nU(+aGQOjnA4v9;9B_Of8nV!4fq#vpl0s$X=vR8i(6$TL@-V0hkctr3#DqwSXAmcJ z!FRfAN((t&>_zF}`wrV#^^kR40|W2_#u@9V)1LCDlp3~`r`y^)Hzu1_9-Fi1y3EmS ze)Xd8UK~M%Z&A(L5W1&xFD@oq&rWGb{+w{2)73ty3$uc>fchgW4e3PUll2y}9;6FO z3HBWF~>YB_Y5G*2RZbm0}AHz>%#Vt5({4#`jGHv2AL~X)W=|;Xq-aFJkhMEwp zcX7{04o~~Y;d1}8Q2Q;1i&_6?8C*jKQ4T3!!7S(tbv_;*nJXWN6}TZ%>qiEM5K91t zLqAz@}d@#I6bfGmZVLbKL@@pmd*A879`m{sE zp{%t;X`$x|QaGsu@kExL!E)*_$RoFfk+f>68qg7{!Ss{FG2b?;6t#w359j`}-CfF= z<(R%|gOt)Jobj>weh?WzRyJvIDJS${y^Y&+VNA@U4U*-;ZgXMg8D+x|NGWnfr|ga5 z_j@B9W1T{yCT%m)giz-YeTnHCt0M@&0OFV!=a=t^xqEw%kSut>F-8sBM&)c8Qy$Xs zobBWdMo5K|$U+_F%kW{jpEKlEyREQYavQ@ufc?g9KQt`#Q~FF-N#^T-!Dc2|QURfa zn*)7~nUNcTfe)PP4xELN#V8>^oz2r++JFW+M{a(;j&5TSjim%mS`tkkkzI6_mo5QZ4`C6i8>%o_+cAJgo z)*0SzQ+ec!mwjoTP&(F0gI^VyoV?G;5TwonrF5)!2h7d$UL776xr~0P!o~a*>3;>e z?BR%+2W~2-J6)&ch#n5EbjrvNu<#g#Rjzwh!@mr#Ul7~CyJB0S)!j5wYG~-MI4PNO z)6BjWiu?)!rjG?{4{o82BC5pwV$SXOGi*`i@I?F#t{fMmI>%i7T-D6W2@-T3-qfSa zAee`S0iY#oyjmZbW1 z3MvpJ8iZ)7;pn3z@+e-R{@$_4LwA*dio*mi%5*ga?h~3?6td8ZHc^|#FKZrNFviBn zap1Y(GM?ggx3dHdl-CF+fJJH$yrmc~DV|yjMIA_D_yawDzZQTdZztSW6$%g5m$^>~ z;8LhinRE-m6v{z;$~{SRoS8f53(YbLUfgjW(lOF?F?ASnlzfL0U`^1*`rmS5wO}AB$sT!Wk3yRS z+BfY8eKV@W+;lN>MorxCoD-rn$Gd6JYUZ#zGSPn~qHISf!u4#3EO7^KKR~kO^fikCn zWHLg%feNI6gl7tA73>)#R%`)iIb<#V&}18FQaG-ZF2Dqll~jd`wQQTSXY}2Q#EEZs zhWZ@oHtmU6PgV!~VYAT&zM9UwY6T)9O-6;%ywN$tLq9Vx`wZSQLj9D=U(>T-xU zFBnjp$?yhN-WEY>w2VWNiCxpr*(3496WbUoys8S!*{6Z-A~KI$P_34MVH2y=2G!5>?F# z-eYXO1pW001+zitd!YoA@imjzPxCHT8H1)=OrV3R0#^iBi^W%)*dkT29!v>=r@~=) zefZtkYY0e#pai1NfdVFIOSNq=Yz|(oMEE)HX&?zN`m!#*a17+cr)D9n@ zD3spGcTEo!&R2LYd9FK&fLz` z7%E7SJ4}@hBeNN;6XX%YRx9j(>o`h3I*xuT?P7?Mlnnbe-=Pss6(r-B1FZ&2lyV`x z{ngkDHMN@50~e{Ey&2WiqiHFWafn?hRYC4Pq@su|isRYp?U#)*_QJ6oF^c|4r~(Ch zsh>uvC$5d4kB}n1H39ru-be**(kC1q3f!8wBb2ANREc95Mz}V%nR{#*>MV4Z@dSF| zwZGFDz(|5wjhDN#(VK}?M_LhLsBVU74&c-t9VQn1x$hX&8jgw1V|TaZ1-;|E@o5kh zh->~w-*I>9J2=Jv;_^W?#OguNsx*i#-%E>$q)cahM61*nA#slDIg!|u@K)tm^ryj@ z_eqdQe(!I$?eMv{QYr1j(O67RWP4_x4Ohs;<4kr9`q`6z22RGQn}IKZbQ|jMby+ZN z!Q9LL0Pj2O^}d$0{`ti*q-0NPRkV!2lB`s=UZ70?nem8&TJR2BzNGy^~qgwQM*pXF6LGto4S+BnB$&D}8L5*qD7#{PcrSEO1Mx5RR2&x5yt--|4fT@} z1yNn(oX4-o1BctLc@tC1&WDxNO^f0N>YrvGsLk_W>ZY321J^s|9(QCg1X$%}>=9lD zEfkCNqQwn5{&B+oFOsb?NPM06AldT&PDig`YGePO)r0@)=$C0WAfUX%rmb_rsB>lM z@Ks|mph%R$*QHi|YOS8lpCxW4@N`r9d}7fwB`{^Wnuh+(G_EHH5dfH{j*hZ%-K0OH zv%TybB-sElsD=^*7L(-$i2yI?jls7( zI1PMA4L>lXN^zQLnScu?ttYdbD~1SsbE=_XR(BR_jB*Kt`gFxJeskkc9#d&>ahpPxE-fv z&Spu;{uyk+n!6x`a=^g^^_b%Ze0gUZMw1oWd5$Z0 z*5FK4ZgE;L04U5e(@h9UO6&n<(q6$SV z@U`U}0-Z4qT}<9Z$*$|E%HR+XbqT8650d!EK64YkCcX}HYe75@^%o0nG9O7JatjQy z`A!gqfbv|KVy=r($t2mP$m`_P_;w2NWK_ySujn%eQv^iT1r%ArdgO8cxHqBI}!z-lGT3Ej7xndd1-7Q?vA1#+09le{Y%oXcQ* zZalJR3b?5iA*U1ng(a~2r%p@6Ogrwk+A#NnTk`+&+ECKwKdTJmBz|kP2HqPooDUtz zARwUHql3=Ah5O2cl7Qg+_)(ywI!}xDKzvpMVKu8$7^vP^|^gLo>Q~|$Ge3%~AVtql30sw2i zzG7c;DC38Eji~&tUL5(G_~WmxI*fa zy{5KKFNtNRLIk7_A3&MkgsXv0Ys)zWJaZyj$4P5TX35h{O}1Ba{sx+a_Ua#Y*`XV8 zjj57<%EM}BgT1j^%*NL~C>Qa1Iveej^puK>5YFWe#H@l*`LBNFpxpptFK+Y(Vx9w$)gp4P)dImtnObg!209Aqu`G9N!)A^q!>Ul(C!ahg)`G z^_yF^@ZpwKnmyx+^V06p3c4WSyBC$->I4`Xm_@~Au}fQUsF^LJpV1p2jxO>Z8howw zn+NHxP0Y#mCE@J!uD(^LW4DWTCMi?~*UB}0qOtCshi8s=BIA&Yg6lGnzc<9BSm&KO zJuWIlLe2GZJ>-z@CndX>vIbb;5p?MRSLTl4`KW#{OWfKhV!LNfbM3<|3peB;eTF7z zC%UIZbJZGKQ8QS$8J^L@${7_oD#M$N%f6vM)|DE;dipV5k@_W}oZt%i*eb;vQ3@Bs zEN`xgWkvj?q%itJwOob!?A)goiX2+mv7Z%DP&F(ZO9S0oq^xRlnAKHACf^${HqHv@ z{lAwa@?m9bN~> z^`EBCNd6MnH!v&W^q{-{u$^k79j&5lvg7spf znL%GahMsWXX9xal1K)%n_#!a;Pa&c4Zo`xN^Y}Tnr7f{^3-U3Z34z$?4ML>4hB^6M z@K~X$Mkl~jDedBxs{~Z-(J|o(4+TGKT%rSEGWJ2G8TDOR;KT?JwckX~9=W%Q@(nhS zKX2XTub^s`gInvs}iKL%TtghVYoGIZJdP^oJgvjO4F}TbVDZ)Bpc-YGr zHIwXGZ}^Y~`c}QNM>I+>=jDhi_9F`|L*zO3GP69k(793l!ZOPOBp2Tu;AFxIMMLH7 zJcD44RD>Je%%fxxuoY#CH@>8Cg+r}+DR00Dx9;{pL)g5}nAF*&O}=4vWc2oSGa9sK z<+o*qLrN6xTUsLAzh~r|4N7w{bmtunLBJMP2Oz7yI(#;99nB6u{YkHZ=V`=SWVd@Q zsWz4&lO5%@yye37d8l74GDGgu_!htOR#kAA&`n$JQmPBLWbJ&vdU@_#cYfIz7UHM^)En#lBE z9=yM!*bKES-eB|{y_ggIlF4G%3As!|Fw{a7D;EKzZEfJi8Z@$6o?Ns4v zJQ4ekd&F<7w3|je>{n)=vfzKlRX$O1lWrq>X7aSF`_TqJ614Z?wrhq}Zq{M6j4J znIIQdmSU)a25_*mTK3G|T5=ccJt3qPbC(!>y-@?RA@j`aa?zBD&4IC_E{#NCP$;oe z{wZk_dICx8rA^QDcG`NI!iaGNM+<%?u1-T#xEC)~;K@_mhJ6wvS>%Nk^9^^(w3q+p zVR9cCFivsg%a7%3i!MTAJxy4O4ZptcZO<2&ebXbeN_S_4tbYL9u(*CyN&Y~f_Y4*Nbq%4`GmWFptUBuzB)^-RJ*ir9tot?$AQBezm- zLEh2Ikb>+anQSS!Xnd@1VMz+chD<>`^1h}vuo3@N>rvi;@bP(!bB z)o2ihcU7#}I);sxqq{3_kOCc~*;nE#)rC`8mITU+G8aYZXoJj2Ob~!=+IiP-!=}i% zoqh3DOXzb<7~)~=)n)R@ly)1(S7JMl{rt=cv61E*z_F`xrVYm3#XK;S+j93K>%%Ql zG}W>5F{F%c+r6CaZ!SQHlWK--Tujeym$9YUvLjyR8C2-%YHQB1U?2q**4?KI4kUcZ zG)s<7!*eFoHmFXzOX;0FMknj(?U1BiMC(802LIYmXmFoE%ong6ky@ zM9)>;TRwZ%6Pm7a=XkK3PLHHd&DHuu4-CDHS=G7Rn@FB8#&Jh)tn%Y3zs1>ca7|p& z!G>vxeXT?{pFYC~Ca#HY*wpUS_J{wp=@*o>&UAg53G^D33X4}2^niOyv&0b_B2Z;>BV@$MT{w~dY=b}x?A58uqVUO5Rxvj?NtCGp_Lveg#a zf2g&=QKaQ-V-K%{E7+VHR+Je`#g|;BBRB^98dKEpfYS>)yYBCp!R$^Q&>{-B#817_ z;goDXG_O&6?Lif{bmGJCO5{VNUkFdNtFyNSHtPpA?KCSS)ZD6rYvd>x%vMdO9 z)Oc{^I=nkj`BJsemMkTUIKgG0FsC^PC%_d>pThOS4OMbb3aJnCF(x*5>V>!a2Z;6S z0axqm1gRBz_DuHBnWe0V_>!4vrrO=o>eZC;z2^k?NK@0da9YhaJ9^E?Q|{nVu2tvF z^|7EV5E6{)Q<2z_;2z|nMN9mQD?!RALL44QeDi#no&dRf)<9L#{zLD?fJMuW(||LRj|8UxJpP`x9?!t|{Qz4B%Xw96Y;CoDd(WIE z>@ME5VR=sJ?2sZsRH~TpO6eNxfF11@_;O>ANDH|R@LF6$um zJBGJ2#N8>n{%)|v&n#uz3-@veRbhBX>Mz>_+&>YP&K^Fs?4~|fBRX3-FRt-#QbSdp zfDFmNMhKnm^(dM3M^2@3i;LOkNt3W9Qoo+8FoBf^VE7ZpHmO>>tlJZGjh~f=4_nUw43YO;Io;w=huf=DT*JTRPCBK%%bBdFHwFi_5LkC(aX3e zmX|rTDm!sWQbuNiZS9xS!;yn?$Lr1vEzs#e3;b*zPbh1oa1}T?@`OnCCa4(L4?{X% zra&SW$=ID(KxG$%bP-xaHj0590jW|enh>Bc_lLD+Y?5MY*|t)kxzF$|g4M^6%SWT7 z(O5vgyfVjvpeizEHbn`T*=7TyHkxV4#CqV|qQ}CuT`e2GSxrWGMw(!i*3QbkdVWgB zF(kROK8HZXtf+hF>T!LGQ;ZFTx9%EOLz%Am{=-cJ^n~_k5z6aoIFv zZ}rjVo^EJ+ZUJnyv(#~tGjeOpZlx~tW9Vx54dhh@EB!_pQQC46wFW07E^#X6WYuBi zgG~8E&MjlXk1FulS2n{@N{7U+vl<|vQ9KlApcYyF;^3A;OEU-+C2z{jU0#LN>{8w3n|T8dACY%Jz9*ejY!{#>GA8%vS+{mz%2z!VK57rCi@R{;<|<{*rFUT?M~))b$>yQ z47^#)=}PF0>lMk?SMLEBqpPc+z`{7l-&nY2Y;~F=;N@Kf1`@g|_s}rS@YS4xlOdy` z1&-+0QP3~|5|*?APX*Ld9r-D&h7h#h$ zzr@<;Y9{tjx-zKNn%>q!>MfA9PR|9r9sBp5S_d7$<0#g1{m`h|WLo25Nwlcil{5 z^Xj}c{9LI)l~ZY7QI&31Qv7&68%wQ&F7pMJe%7_i|H9s{Kr&uZr+B?3G{gFRI6_wG zHeL^s0ZUWKLiXy5fEs0Fj*(Dbt>x9P1UE|&u5BPyDiRlt?UKXgaBAH}@l|xPoF5g% zPX<*5Z5lhnFp8dnu~3$T>QX!3DzC_C8gwsSG)kkMgSd~eJh(C`os3Br`HQI{*7%b; z8Z6r}o?TGzM#!MeV)phB4zkRzNCCm4j<5>R*ulzg%az+^j{Z0dj@!0qzf2B8?zi}N zZDZC4irwZS;hB)&d-vg2uUhoJZ(vZIKYB&EO!T=Awa{$; zaItbsQZ7wPtu*+RKwMg=o8!J7mCHR{V`0XHsr)0zw5!H?IKB0O0KMG<#LE;QQqr46 z5`Hbx)~$n8soOx{QyZ3N=6%Z@Dv4sFU!wk2Z$elp_y+PvVYu~KF3b72muMRU2uSIF zUl{&c7V6Z@Tovcgy=9DJ(7wDxze9aO2p$82lufK(A6=EC8ST&D z=t4s6&9ARZ6I$`jwL(U!10{llxs{HjZ$_9k|m;^0RMGBtKsKner-na%zdOvu?fpj{?hjZ@8*TvoBndIBD_kgps zn&zkO$i{%A;Je?^YU9r0px!ARqYk`>oBLHXzl4Z9fzb>Bj#C126&$9Bp^ltw$W zm4xg`9Mp{^YdaZ0&~%{3x!sM=Nf8ufv{Ops38RzbH{fsu^2KB)nDg-mw9Hmf7y~!8 zk!OE^$ra3!r$QG!YUGYTH1t)vZ(OWulCL!=7ck@eG5#7;E9QdY9MgdBN%$A9i1V+$ z>b$(=Fm_f>ykWw3MZ9mFcN85uy!8Ar$#bfdRTVUr&s^$)H}xl(&SR)wL^#h@&q9t1 zWN07!HCzt0M<9dP!%&!A; z6w_-k4s^`=wn#zOq)MTyF3gIgWgxhRQejBV!XN0A_btzGo#fNOag_{evZ9IqZJ=tE z5{t&J@OR(lDivtUX-R=d#5n2FH9UfonV|;C&5);M&NtTj%npmzl&&C&89O%dJ0AfOZ@)M!7c2r|}X zTOW~({j@K0hmr&gP_a8omH>@7838Cq9&RN%`P-he*?EJbW)Wye+olOsxxoeD6# zaBZ=_inip_P6@Zn_}=C%$<3t*jot;b10!i%N*l9tsGZs;dmB-+xN_`lik(WD9q=(gLq_7T|3qia%pkFZXhgW~lUkqukv#$rk8cFr~HXjvU zuG<+ib4sRkNv}91h-Srt_H6ykS|SYz>O=}$H74AJzF_8mBU1RTl@!ayADbCGw;z&K z7Q{`bDTrZ%P9 zZMUK0ve3Q9Y~HcQS!9Yt{ghA#Q^hM^%kexpT|SKMbz3^LFH1{YN}|Y4joF(*$h2D; z6z$aDw56_~l_if~v<>S1iXI8aDIViOd3LmreY#p& z1^Y2egrMF^wrv{m)-Eo96Y})4w{af}d3RVPNUI!gn}l`y&@8A)Qf`q{znO85O`pn& zHHUXLt^-pAHz?bfwhGouAl8mLBb&u}G&A0(;h9oqt|`{#}Ma)X)oq=>tse_ zR)ElADL*uu%5Qm-V00jg8&d1WUi@5##Mo4P;fz+RwkfMV5=r$df!pFapwc`-odgtH zjN47*q$^m+32O>%gEO+ZzYbWaID7{O?GB{do98B{kcNtKueB=ykP-E5G0INao4EKc ziED^44ovC_vYT*Bm=0kQQmEllv#J-f{C@Olz4Bz?kf0RT#8jQ+j1 zJ>pb8*tdv_VVPY)zNb}Dn)=Mxwc(c|u5zm-CDsMBrkB|!TFv#|1VyFcW2)VbYl{W{ zKnRFCZ*{v5+Nb^#S|YPw%wE8brraC(IG@aBHpx_%Xj_F;G$;D8TiLXy5_PUK(h3U3 zwx7(8`j1MQnM8I-OMQ1=I?1$Y2w%lasANF)-Pq3*vv9sjt|d`bL&hS9ExSxAkMa>G z0@)G>BsQKJ4R)hn2i37hvuGurU^U?xLQXh;6=AW>imo}wfoRBsMjtq0?i zgjTYou)`teSN95~w(bC49ISahw7a^FXp6<`eZ+Q>~6#y4;5LK{y_ z8olP1PE$?k6rUwxjjA}c$sGboN_Pb*8T>$kEiAU~bXr;%GeLCN{DcIR$Xdv{qYwuw zol~_1alO4m?yK}V8(6K7>>SR(YPp1pmvn}Mw5>eaYn2?dBKMsmWSV<+o5pfrZ85|H zuDfm3DXV-#|F{Y}PIYZT0oOckQt=*R($t~@RZ3N+oU-@@j44h7+N9WL%R{wH*WhQU zR6V*Y`SnB3B1+I;pm;Z1G3pHDNI3h$t97f*+fSlpL^0^aC`-;@NY}wcQ}b6*{zyx* z_zOa!v|(Q-Pv1UgZ{i5wrvv)7_SvO?lzfm@=$dCMbzKYR=u~b=32t9ctZFvoJ{gj) zGD#iW#JJooPcGjUrnx z7aVt%t&vbijRY{Y2V__(cp_o}fEF9=;lQ2ve7G7qOcl~oZE0FioP$%naon9@_~Uz` ziU9?(KDpxuCTh2MWskr{E{O|n>ywKD6rn{3Tu021NFoGmNFu{ZNUPbFzDQt^Eh(ti z_pKB~!(~09vY$+PmsDn^E*%*)mg-vM_haJM!{V3X`GY}fzS~_O^XJn^as*bZvd#!K z+TU+n(LcKwr(h)vC$oZ9Q%NmyMUKqfVs8Q_9tglE2w_dkJo;g1=$N&|^`ixlfNL9i za2|&Cr1dw2qv_*fE}z36qAT%UZ6q=A?1><|+x;cRBU*OFMmd30im9CnScTQj_*jMw z8+hCcmzVrKE-6TEq_{vU6)ITdIi0NK>zl-6A|!BtM=98gCEw`(BkK^#zW z)MY37XC}h}G*-{2t*Iz>Y+#39B2Z8drRr^_Zc!oERwOA=Yo}~z-3>$v7y6j8cpcffm|?KIb~xW>uCTCvO<_DFB$vyPFw7?SLTB7x52Q2` zUq}|@Lbc*v`-mhw0SN83jTK;}TSvk?S_ZyCbp&_@-xb!F;oeoesUri8= zs;}YJme2|k#;Y`X;62ibe2MspT2w7+_f@AxO9Q!8rzUKj3{-l8-9_KvV^|&QPZGRCBn@F&!r=l*Tq?a_*_!C}#jejeQ@D`V84- zov3wDz`N2H&U@&urxB^dLp;gN%0vk#+DNRRt2A#v>S^FTejECnF2%K`wlI07wpNqe z;o@xWwALtmV#$zC3pceIXp(FeHsH}OHE>I1KvG;#*Z+FiUi$1ST~mnF<7|*E_&l`Xouyq2|`1z|%{<;sRZUdL)_o zd!TjOj)8eU)STGqbEI!@r>?3%BZtd~7hJEy(*gZSU zV*pF*3(w1{jX1w^B%Sn8?grcSCV1LFO7%8y7?xU;#OVY&;%jaxu1v%F35WdXIK&wYoAn>NBs<9RD(1f*)L~xIw~2oMA7Fb zu`R6aI1)Q$l2arW(nf->FQEC3htLi9hcl$U;qd7W!aa>QUEK_m`Ix<;_)NV^xN<5is zpj8uvJfMr7(86xQ1!Ar)D8}`x|J_El)93P5?+rWuy*_FgR<{^t)BCXi*Jrtk3Q&vv zZG($#P?D@CA1r1DH~t@-)9+I|+dpyC+$=Z5LXz>rI*;9m^A)7CBP=JyQ|cv_g94Ij zW-zAVFy?jJ0$~PC2e(0b76ppbF{Ha^ql5Chg!y{NEtR&+FMlfa#9I~^G5VT8k)O(e#*}^;B$g1%9rpj>CTmMgbF8 z$oRS!Ffg;a^ZZF)v4lf_aIUK-FrmKCMrFYE*!rZ2HzXUiHC#It-N98G5G*-2-4a|Q zci}Y~{dCYfS=$2*8gqWbNHI=dmL2XQm?%-(I|9+2=n3?73iNt6Mqn3uohkz?l3}*V zJF^HMOlcgR5Is!1(FOFH&@B+`vf5vvbl5HOXNB*_~M405m4_eyKfSf4uB8 zy0K|((6!pekduv$)wawfIat6DjvTA5a4q&Y*(y~?jcLBo3OaqWK!a`QDO2wlvL}I< znKMa11yp4BjC{DIKhbqA7mEq<$p04jDL+5KGdG%)O#IhZRRloGxkW??s|<^ux-pQz zKrXP$0`GnRUt(?j_3*Lwi+)Bx>hU9%oHkckFl(u zm_BZ$*y*f@!O}G`BdXsN%N7+2DHG)iARCLGommWRCb$F0L3tI$BL>y}oUg8TI7^as zK_0%Wmr%VXS9Jb_PjJ%lJ@tnOj1K-ITVM#(l#7{qb^MM_b`V7Qxds*VCPnw!F#8_+ z>?SBvnF5Ugo7A~7dj)E~5PPuPE);pqD2z|kApFxVl(HrfV?05xlF4qfW z%hE*fPn|=<7fQ=}4N!ML`5`aq7y%G6X}Vt|jXnQoE3cKKd0f1tT~kj|zICCAx?I2A zqJ!XNDMzHPjw@UTK!ZK6b&3xMd`gHelQOy0Zgm#(k z(?es>*kX7+B0m@_b@!;F@zJsGIljN{glqJ(Y#6r&@(#H&*iQfx zdJuj#G{lvg9;A6rV3XUJ_eL+JQ64Lf4ScS~Ms_1A%x_Nre_FfBs4TawOGt-wcO%{1 z-QC?G(j@}Yosv@00@B^x-3<~F5&}x+_i_@ho^w3+yD%7o_Xlguz4pW2d+legHK#(~ z2)*bl_k=}x4M7?9bxw1y2U*xj>2PNJnL5Npb?)(uAKO?#FFJ#aIbGzkKw96b2gb=) zk8n^qsN4Ujw{epNz8(YY3fND9fVlqKL)lMU)v0MIp@^Z}CV`n^_T*vvyg^_hdtRQ- z2+Wd{?l(|gYh2e$11*}8`Xat<>8%qjWL@Ib2I;{|DP6_5eq>A3T*0_Lp|r1{U(E_7 zH;(tpAwlZF@pG?^_ftw*94@8LKHlAW;Qz}CvEn^7Cbq{ zq_EkX_2|)EQ?N~SM;QDRJ8obt`<=^^bVtKEiZPveZM2HxD=IW5`YkEB7TV-r4OdE+ z#He;Akz*FT+K0_wPGF?r{j@c)h&cml6dn6wyG~+>L8M_)V}I!Flgq&l)6L_s7wDB7 zf>}$w7M`Z6188=Z*hkedPNV}(&0suRyJ`FMD25XY`Sx`&tNTec$}71n&!4Z8kVtHs z7caWb5fMhg)Mstg_2ABnzuMTuJdJo+Z|5OT7b_lpg$Ad4D7~iHP8~jhCswz`6|YWW z5V6`CWV9(0iE_4sH!?qr)Nxj}wBTmppPIH6V5Vv9{cK^ZUp3Iz`_nxCF(Nx|dyR2G z#y%wm)H^2ooV+(ZRv#H$+ffFsLe@{GUwR0TuJghGOA9WVxFr7%A59%1gWed8PM8tyh>4seW!W15|S1TDoEpVA}s5l z>7g8dlm0+&AepFUi~AC)+6Tm%9tY*voFjF3Cvj^udjDuhJ79uiEP!TQP5oQavlMyN z{WroiLO?5+Ec>qtKp-j+pYnF#8PpSuhtoF#>9xsFGp2Y5@+BCV46Q=PA7=PO9x`PshFY|R^h}vAjm$KSb3dOUYq$U~(&yBcC zCKTBE-wU;~!6u}spc$D5kt!w2DFmL6SfHA+aI(dA&!7g*ba|`iymVzcw4aVUq!BW7 ziDj4!4TzhC6}gdS zn-k5nG(~|@hmhm*WVd+M=j$EFJd`1jl*=>8ID4{VCN@iOZ9K8vK@ph=5ntTBrMKG= z7*wRF@`h|J2hv^lSSs7oKUxpKZr!P+OrW*|>p}o^5JoFV6p+2O22!q;m&!^~(3!A* zy2lEo!Y*p^VstKGUHMrT6ub8{e{=ww1saVh4|7S#ME%cU5Oy2zBt8^-8S9_2eb z>d9uGB(A(0%ppUPO6bpDdHjf-sG@w^cu0g%o~f3q71sK0<=x5cX|>&=;+WsWDH}a2 zh1$z0F`Hm39b(heWUC5+eCzJ{67@v6h2IHu3L=4Rg`*M|ZyW)7*1QYv*~qx{$6T|q zmqeAw???TgWSj%voI?qngAuEDv){HGw`vMSzeRd3Xeq?C|B0lJ(Q2Z2?Ai z7trBy)V~k3=jNJ9(y`?mwcFXCeDHpm*3zk=^La@qyKc_1bCQQ}-aLM8jRm0QIp#!J zEZ;i)lA<$|IkZA2{Pd&b90i@YT0BP3jzuiUtgEae)BLHB|KKS@^ z^jSq2ITQf|k1BlFhIB}ukRT%3C)}gL>muQO(3y#FWTo~i(_9N&3&y#xK7XF`!(WX& zGG1%LUx{R#4#NYgfIxi8GWD&X9?&-9@^*cf4M?s`Q4A9X_zPIATwa0kiX4=qFvWWf z!C5N}bSRD}1HItpJxwkI4<2QrN(sa8!}p-1%M7JOf~mddU~BBQr5hlxKuJ&6&8kvngG>;)FYlqQ6tc+;BX%Yp8RGA~j=&jA;lk z(khcyibKjB=}A@)h7@D*L<%ijKprdES>qmv=@y^v#wRJP^3-QSFThPb<9(Ge-Cc$w z+1~DH+9*3&jJcy+(VxpOEmcxvkTbxIRlK3BOzeH+*!(UuR(2;9_6Ad;q(WDYLpL(I3<$10kP1!A^|h0npsVMF zp`Tg8cmQc^RLkq&A|K0UC=8)iJ;U`leZ8$1{ddGuJ$n4_LSc|aEa5%Z_WRiK#9LVg zOX=24qgg^ZPLXjMI#^%(HPp7XGFIrAol=@o99bmQSsNhMs6BUgLVWtH>X)98f8HX^ zg&GQm0}d=sNPk-d0^q9O^UBSB<9|QqWB&y|`zNlJrT~d7vI@%GlJjRk=YZ8QkgpV$ zoxJ8XCoBu zyJo|2;&xFIl$x-79+K3voQ=#1f^ zq%QII(8HxSv88@Sx!svfW!VXl^UKPz^ya>3WmgW3>L?-su8Yo$oOFlEs@odYvw%!C zX+p3iX@ZBi$V^i3xU0RMD9-OJ3Yc7az)_CTeO_Cp?Lz_!{g4Bm%ZvABwxuR10-tN( z1Y0<)HHlU?-up52d%U&^U!Y9G-in~S`X()M8BM4l0t?7<(vs$61CZ~YgvZ(idBaI& zh8ie|)6_HWAw`5x_WL2xedhGpk_m7Nt^0M{mJR&p?^t#0srUD~>;DBYI2Othd8FUtZZ>!&SO zG9jt=ID@)#sfDJ_)F?D;)6CA;d0k`Ze&US67h=Yu&g#(U=(nWy*R9RgN zvOc%U#1ft%R851No8uN30WgHSH^_nSX+vYBE^ThqJjAQPVyd^Yg$H9zHiAKg)B;lk zwKl!t`sYN-O~q?97Dn~?&1b!ruuchz2z>`&k$070ShNLXh4A9RvBBWf?D$0#d;>wT z0I4i-vc_P(t5`>aYb4<{i8|cOA+K+HJZ-}Egm;r$X$Yw1F5#d#-KVrk-@75kaxyY^azs`(eqfoq2QAdLn zo2gV(S=5%|&?JPxm{&usM-{qJ+bMg8r-l7_EYZKio{G>C^yW;T4&xNjDHtz|lzejj z6i$7&tqHE*W71tD1&dihOKiP0KpfsSn;B_jjidE_w(zFX2(cg&$q5L2`t*BU63Gm@ zILEap)@jC)ebEJ-NnS5T$3t4yMW23aVsYc88hK>L54X>s@b(}m2wd7QlStJagSiEb zTDElLVsZ25Y0c$u`(f3GU+VPpSmpLDNnQ!*Y1XD<@Ujt`3_A$%5pYu;EcV2683(Kh zEkeXeOJ$~o8?s?{FtB=KaEWbgO3dyFV!FZ7n2SLGa#<7dMXv~1dqJKT!GX4kDW%vN zSRVzVBF8<3T%vGJkQ91?A_xw#tMgqsa@Woe*ac~BGOILGXj?#bFj1GbkM^_Pt2eGe ztdpFW`%-ftvE`GeE>=!PKVf91Onr&3h3^a7zg>#r&Fs#9B{eZXpVUw9JlNrC4J*Ak980PA z6XG2`2#T<;%d#o$O!t|r%lNZdu#3&SV~frObA?Y*8KRX#kza^LjE@ipIWNBSdVhHG zpLCQ-4`arU5I{g9s6arn|D!X={r%=oB)7lSP0%7?k$@A^FB?MJOXCpwnIHqouxc2p z+X712yAln_UqDzOtczp4WCmF;{`{tCXuLAKZw;_kno>IR&t_CPEII%RC%kW&czBv@ zOc4yDUxf8@B%J{YCu$C858&@E4-(jbxqY;FnT-ZR37C)A$yY7#7cr0rpr9o1NwPa5 zXbEY$zv*K1iqm`KA@%t5y1itj#Y0{@@$r+Ob6(B#!@n$*NV!1t-0P4#R64q>faYHB z7&(GD`Z^$a4Lt%~u;uQGM+a#^cX^1Uix#OTbsqk-WJLwvd5s0C1ct07?l8dbBcf&G z2g<3@Oy8b)E{O6zJRVf?UL8tZ?j(6eOctR-4(xrq<(Nuk&gZ8th_c7qqTHF;aodSl zm`aBAX%S%->y{*0SPWHw^dUtlD>uGB`{ihOuY{14B^my94Qo9bD?Ac+KvEx>bo;Q1 ztEHQ;I6fn@5~f7n+PR-$_9TY)Faj$p6bqcvgwk6hO;Max+z~>WoVOpzYJG;@Lw(6| zlO|O@GpDSgOU6+MS6U-j_?RUY*J;r;+SyRC&Dz6U-O!Ney@ecYWyOqClMzboUs_~A znq63-wN1Qw#=TqRScj|4broSp8m)p+LDsguGRNYk&v7`kn{5U`0=2(e?JGoGgdwTO zqT*}E*Qnb7sREjdi=kQg@uVphQ=yZP^|nT2+?*KrviueYEnRdvB=^wpzv(w9H{ey97-a z=R4(Ea7w6l@q9ZyN=<@XEIU<_hv)2VwDoei`RXl2> zZ@=LBoMajwUCDt}M#&|@!JZp|R-$mGkN<&;;XTPWLT-EfcFe7?zFg$fMVJ#rp5ze9 z?rfPIMJ0Dc6`T;Vln=dZ>1N@1^ZCIc$~DwQE->0AdmWybug1t6)W$?->w}!H+G%DC zFzN}{2@*FV8gvtUHb^lM_Xr*1H{0XKLY9znjT`-Ipg`{e8Ohe|j(c6jfR1Qs2hM2& zpu$}wlraxhJ!rp#_JB+K8XbzON_7g`-SKw^o{Ix|(7A_EG2Z&|Ilw`RpNEFY9p;CP z*ow4oG2Vt)V9FVax2s^z9Quc0YQLeUA`axhoMGtM7wFqtrdkaMqhp{;IbmzRy;`GQ zKd9|{p^-mHgj0JrNz7e6T5n>WnWVyDLhoO23BUwb1i{Dy`d?5F^3@%4V7gTQLlp+dCOU4)Fj%s8BH`q?!3g6$c6UtN26_Cd) z(uyFtaSwBJiij|y-)?NYqrypewX5Y_(x<;_VnV|xb5c%fJ(6h5FO`dIiLa(f8z8m4 zmyPPt{6`o9{uQZz%%=4DisWK>ZoKU<+}GH+ydia)`E6qXAQp5hEZo)zN(Njp^)Wn!%fedjrPwm zOKs+=xt!-jzGyt1a0#NAYhqPQF6Xh(#1Fm}xEl93R_x+EpPs04bPA4|#Zh8P%IN-j z#%pA3$`oI2R_fbwnhad4KxS=}78lI5j8*3Wbgd-<(RRQawf*e;!pZYh?3qMiz$QQ8 z40V68CiBNq*?8dCfaK*B2QRC4ooseqVo10U?+~9_IMY_1`3fd$lFc0AoY3XL$wPtf8#y%RS^rE45;33W_C$E2Gb))7rrt+~hch zgjWj2-rlvxfK!b!AL)!^AQGYJo(JDUjRT2$y(^rMwZ6_>&XboWhPqpoK|c9L$Z5cF zc@a(!geMZ(4Orzklhmm~W=*`vt25o1X+aQi@hv;iFg-*@PRj2Od>IODA0lBK*lLF z-agDwx)t;*!9t}umdPO_Wc)dtegi3ZEr(gvD4#07E%FcTX`H_XXewdhjtOtzt+b7bOI1>Iib*~)Rc{>nb zfIwGYXA0C7l{mQucpEo9E6GwYIH zu@UwXMNn}-BO~0!_WY)B0vzuR1U3nGm2lR<>XmFhJ*FhE3Xf4sRs#Oh(;=fRTMCrn zjQqIQ4zA{7q}9AN>iDvb_2F2T5L?Il+|q;OC2~BTKwaZzFro-1@VvvZ7$e$+aY=Nl zSJXh>N%J}!+}vP&9$?lp=(aRuiMUV}1L^j$eSyJjqDA}cn$&X}+Rt2$C~j-8gcC{k z{BB%=q_%6|Z944m+>euOw-x?_x zhAb6J5Jk~7?$&d#-}thk;r4R>8#>TgnT80|j5c+Hx?P!Q;+vyZJfv-}m28bcg&29y z!pd?vjQQC`^4Mx=F(gyaoQ)6E)&tq3^N7UXyrD!Y>5$@HqwDvbs_d+RPmJWm2ScA# z?{YP0;9x8fyTT2vsFI8;=lWR+Ouic22P6A@VMx&4e0-H@G}wCZA>;U?sOR+O%ciol zyNWMO^CeD}e8%{Qxh9q>)P*=slf563sG&9@`r+uLdzUT}4w> ztg7lJ%r!N`S8Z#-d*M;jt6p(FEoXOD6q(ZMF_UDIv(6?~$x>YIZl5xIo~@qZTdJd8 z$T+vq6uR~Wq9Hopb;Q@#N|@hBmF!QF5mk!0c_ukEFcMPV#ygY=xY{Z1BP5Z#sQU%> zpc8Ymt178`ceF6shV-EH_u`{ZfVC%p`C&%5r%~!T>>_Wk2c4el3=6mgPT;&+2V?hj zxKQZ;8Ak7O2{ASszM_x5ycI7{%I2)4*K|Tma7{C?4(^Eq)$Y;&pPoIWwoEj71uyo^ z$_4R5ZKJM%De8<=+`;{$_iLFx^?%4jKmi^-84+ax8VOm^U#r_=e?|iW*=OkMI6!@+ z2=D}>JTxIAAS)p%qNGeKBl?G^!%Y2m&B*uD|^k8&H;n`48wV*zNeo+eL1L+3Kc_$}tr=BWhI7 zfZ@2$=IEu$=$rJ2TVQ+Y$6r)8Vt*7IBWEq$acVA%&!>GCkC>*2euEifxctcruKR1x z1=Z&84YnnMyn_wFQIEgk+X*zCv5ws@R@WueT~L4ici+qFR93D9ya2ok)1?L!JUxt$ zw1kj|tdhtdQaeA0&61{;!$LQz=UK_DeXf!Pcy21Q1@=~O*{6UaAq(L;YqFR?(o%vN z@X}sdf{>dNo`fdvY-5*8Rhe8CiM_dst*@yEDP1r}buA)v1F1*{PWhbgb-BNN;)`{f zcU;@O*cH5%an7oQI6B##`N&;o&IWsvtX6V*b)!Ab>R?n;g0OR| z-ukxvIny`tB&eDg8DdRRE{0YK2oo)GW3}LMN7NFesxm{z0tD5QX=S5eJ#<|ZUM&kY z^(M@tSA?l~wo)p=Ljr{=eS*y(7t>)2spqnZSiI)0!?8vm>Q~k2jO0+LWL}rLHDLSA zx%5NJL!s@g5`1Fc3!A#jy=hEwpbALEd16&xg@op;<<`4EsKo!mgt%DZI2g$&5Q1bs zE{aqFO4AE-MT4%p6;xNC2*(Bv9JkN{ny?5OA4{>mbz-H)NUg1Wyt3$XBgtL*?5;tN zy=zCEs~Pp@vB+vR1cttXvDG#)COQ*QF}pQEy|0QDYSlnz8iz>FGPrJC2*HUZp=5iu z8c}Y8sr5yuxaw54xff@`H-@QHgKlKUj@qs&OGxJGOir<~pt7~iNBtJpo$JLUs*2l^ zn#xH-n$J&94(KenKsOCKAOzM|z)LwVffkM9V2}hpi8|@~Df&{8k2CSa47_Gtm)368 zyiQ2f+0+fZ8zp;MkEIZg`qJGMcn7g^3LX`>NoyYmjJpey9Jp?SWOBMx3yKo*^Aw(# zrj1Hbq5sD5a~ZTdKds8#aa(2y4OE=MhH}h4lqkPUCSP;M4|XnxSxhITJ%{#{gzPyW zyMc?u0=lq#@_yGPC(f0}T(vn0nC^>Kw_q8z1@(LI*SNJtRup`e&o$4>7W?2U;h{hj z#yH#+1H3YAu#onsac%m&NF-?xwmv#n2kkZfBWttkmQQYE7y)wvO`8sky z#!x^@V|inZJN><^^}$`)+p)Ev@urPcIbhYbdIbfv=+*$&DtIOf9mAyv>A3AF=xZ^d z&1$9Jh)I-;P80f|)%tfJO4odm%n{hQB7hdx8{w`$H<45e$h#Bca+YK%7ur4z^9;i4 zezq90=wDa*@w0t`H!%BiFD#(97E``tYKADj7Vc?~U}&tY?jVTELoZoh8mMo?`ic|o zO3e;3$D))<+c-fqq+DL3M1smr#T?UhmR&TS&?hw=h&PsCsa7fWTq`$KqEMQnbBcZ_ zw1kg>SR!X!1kVIp&yFdohn1ZH60VLXMGYqU5IvShE}7vJn;$dtktp^G>-F?{4+nx0 z>jE9tD7kiCj>1zE)+4Ac+l{y4QO2{ZmC&BAB`=7iLVy)Mwoc1>5vOP~PQtw1!&-X!XrDyY`sB^ux8)cUUNx-EDWa57eFVLg;shqz z0Y_)!rxegsEhj>?hhWQ_-avq7aZy$mlq`t8g>vRo>e!d|}$B6M(Sitl}CARIN zy`%DmMNjh!9=Jjl24ZIKo7KA8K<5m>W|vPdJ}p3L||u1jeXN# zIsn#zFyOjK5843p&x>Sr3g>gf)5;6o(BEuR zy2Omjo>~|-yP-mbo!_>6ZRN~fN(9wcgl$*hyXPdUDKn>`#s6+}i~`qEFYO%t3(~y) z1f?D#@U2(0u&UfZ1iBxnUEuBqn!FJbn}U>&hLOdPRo1leESG$zg78t$Mx2}`qgtv0 z)}*)61Y6i=p~|I1HPzj93C8r?q86f`XlzD#yADC))@_^P+7|7VUy|#6;Kj&L|2FDb|n`eEL?or1$OI1kjz$Csj0HVF&zSG zRp^&W(a})QV}v=$eAtl$!`lw9`R56V*jZ*#0{tohkTlAe++;~HOvCW`wnq}OQYVw< zQ=~qs$1OJ3qf-@ALpNXdPhj61*b%%kn!n(QvNyZ>G#4k(`c|5wjQWBK8C6pbiZ>SZ zB$0A^Qlo;1LL%jxsUdeBBIBXJHe^@AEDN&3a=Y2SWE(gm7AbDE(nahtox~h;CnT!} z@?@+)kMXW?hJ5ifTx0B(w97gGh%jA)gMz3&P41)CDn+{*geDc=%%>Kz2kBFtD zS3Y;=e37V=Q`+=fnN@E=`)jQ|m#;yC*-xgL{F_OTp{tug#Xgy3dP4VLtK`&MT7kYu z6_y4<`*P>Q70D0|ixYsMA5hUJJn?M%Z6d_%5&P{dz1)&k3X8(dv!Lqn(ln^nu@x!@ zFzBPLUX||X6wzr;gE}(|cdY%?Iz~f=Ii2YI5p$*N%KUr{Ztk&=PZM>Ck|a-ShI3y^ zc*iUhNT5a(Vw?2|=P*Q&8NHp&$8iEd4s@3qifDma(Pb&)XG6#5%*%{O$;C6sQZ%}f zLaKmy!~T&Sr=r&?F)q9;4M&GGHy$%zM2a=;ViX}8;__s6fO^+Qh=iO!~n~f_6VDyVFj2{ouOJ-Yx^_B;Zt6r zJYuEDfov;#L$8BTb|%*R|<_2#ay>A?-h0##4G+gg+4vP?Z3uSUc7e(j?J&j6KKkPOOJ zI{DHlG)~{EQG7@$$p^L1fcH#C@dU)FE2GUWG>I>KeIVF@&aN0NichPs%bo|Bk9vB* ziT+E^#!Vy}{+F(vuB9bIr?f}Wqz74@5dIepkKX|oqbF~Xn*z8uP-PE8}eCu{IakA5lz^C)E=?1Y< z`?}PRM`$NZZjDhmT?D8KpD7yr2I8$=g?*j_X|gOc0@e3)+%Sez2R8S>OR<=+qAgc zvI~)23#Ngxvjh%zc(N{$gQ~^t0NRlR^_O6y@2m-U)WO}SMoW<${baeYtNfCwA=3%* zF&!-z%=Pth>Z#XWxG-RDboY&zt>5?rgA5;IHxUYo^}n#gLp-2qj|Xk`UfFiOcK4)r zlkJCqH}M(3lEJ!+xaAM- z*ZmfIJAB=8z-3dt%c!VZYkgfbp^!r~4D$ke>7|)zZIT=;ZCVrZQ_a;f><-`tmfZPW zvUj+iKABdRx@)lr9Bn}%Gm30k8I?Chb>2+hp1}12+4vpW`=XCv9xiE9OBO+oZadKXM zIJaBfDmakJzzvoovGu7u)wH;E!BWH(L|zQ5zgXElD6JJ2l3Q3b1j ziver+J3&O7m;R-`g9#;?E8K~6%`M}=0!F?^S#w6@okhmX7;{X>DZ^qmm5wsb*hm~T zEod2p#dlcV%xx5(rKHK1GD(ExVlt^=uca|wk?An;8nrt#$Gq4vPnMh-Ek1S+jlsiWI}2YL(iTngih2QZLai*a*hJiL|I|=UGs)^hp~$4 zuzQuLsi~`tG=Yn?&{^B8MsM(aOoy@11PXFHETQ{U@@3T~yPtQNwC&>!IxWTZbCd}S z!SpHc@@_$FUcY?NXwD1G54rXr&`(IXd~d;RC{cws_Y(BX5YNpSS3fKfK0(eXIJhjA zb)%&%mOcwSZ|0BB5;#Ag1#{Y+%;ii+M}Xo#%ED{r>g{)SW(%!bgZJ8sJR*usa#IDe z1p&M|q02kB;3J2lkR7pKGiUOCxp-u6&XRZ~6O*1G; zTz0=q(36*eYdX6)aO&D0Y0dG^tfiZ5fMh9_zrE)Fk=9?x~Q`j=T&MRI} zXqFTp7u|yJjP}s`e0XEyjDqNx81jjB^YuCRsO!=NYiv4$sB$(tk<*Q-&;%m~=A1u& zyb$EsNR#~~UhS2o#|@HlE~#Go=({z%+f*i$nD9y2kGAbnzSrLGG0yMATR@MSj z+Qzj?Apt{2FX@S@o3&=ksb?GESF?9b9>ZPLAn2M9@1qp|gkG#TG1PApmE)oatU&!W z^QdgHG4wXdZOb8o%D1WcIv_JIt>-2xGNBg_S`|0Nb+^zpZ0Dl?V?;ihPcm1b2?LQCwdpXMAf!xnz7oZd4 zKLWY$UJ7t@FHZTx5&!x;elJb=&=C=!kH$|p`hCE^IQ)TD@C zzYF@Gz#hwRJp@LM{BuG7BNUH;{Z7jHIT(*6njXS<10W0i%-sJ4jK30W`Z-vSwQe55 zS^<3cf5Q3;<(r>F_E;+9Au=wU|2ML~5Dxh{WRLX~9wJM}|9>O{vohy zqW?Fr`xnFCDXsh+t{0`h0k`Ok&yaj&w6;6mB|=ivTb+#a{ZcnGaQ z@V|xjMtY^)1y=drfc;Iy;9ovlk7?NtabaowZ*V;%Y5!cT9&@-JV)8cn-(mUz zqwDYB{kmm;&-3{m-u;$(Z36U=^#d?>&-(eB9qm5Pe$4s#)f|BS1K82{Odptg!~gwXqUpJ!f*)k6;>zO^U{0DdDe)xZn_xsDdhgQ*C9$5YTYVWU%f`J3Z2t)z+)&t}k KGk`w@0{TCAtviPR diff --git a/sql/hive/src/test/resources/TestUDTF.jar b/sql/hive/src/test/resources/TestUDTF.jar deleted file mode 100644 index 514f2d5d26fd358ad5647e0e75edb8ce77b69e30..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1328 zcmWIWW@Zs#;Nak3xSr7K#()Gk8CV#6T|*poJ^kGD|D9rBU}gyLX6FE@V1g9+ zP&U6PT|coPF*zeuzqlZ=C|kd{Fh@Tlvn*9VwIVgSv?Mb>Pv1Q?FSRH$In*V@%{jj` zuf#|%IVZ8WcxlK>KjA=;e|ow+J<2CuHJwts5Oyj+z|m2lF^e}kXqra-9v}VG4QH-I z3e+&hyZ7o`e8K!9;8^kPO%YMTk5r1!#h&{#_wB!5f8VM%C>{yDve5geZOQcoVzyjW zs{AWm4sj`IiIjvs-uI-|v|~|xYaioEfnb9b2_6TT+q@2c)9qham^b@gU-(}S?-_u{zB$yU{&~h%g<;U5!%kr`i_rG&( z1=D`Ixp}xt-d0_A?wzF4>zUi0YDp~H8)9C0?2p5AdFR)~+a_PDoF*OZbpMg&?Q^f9 zpRKzP`*6vT1$oc^%$hb^HO-n;rgQS&h#!-Fd7l5syZ%O3Q#+5>wT-+Fck$>h6k2#X zcLjSfwFen_YXhGm z^=`M1UD-x;p7>=k`xz!GhN&ekd9&Q?Yf0SWJ!;Oue2X$)dsxC;)*?8E^F7HIkJ4tmi@{)@8^X5ZB1KR^!8JazLNx?@F#6Qh=oqF?F^3mU?`(NA31m#>^_H^wjyAyleIZG~N zN3cs-gj`tdxtaav#WjMwmn?hLF23F(t;XrL$Fhh;j8kUuBZfwg5A03sEB^S1^O$#B zQ*zkSJH>jz!pVO5lRCTRbb9WZtQMslA1J9HGa0HDESi2z@s87s ze>SVae&s**JNU+?Q1kX**;A{PL(Yr;aEf<6t9WJW3-OqVd-SV#-d Date: Sat, 3 May 2025 11:02:42 -0700 Subject: [PATCH 05/19] [SPARK-52020][TEST] Build hive-test-udfs.jar from source --- pom.xml | 7 + sql/core/pom.xml | 4 + .../apache/spark/sql/test/SQLTestUtils.scala | 16 +- sql/hive/pom.xml | 4 + sql/hive/src/test/noclasspath/pom.xml | 88 ++++ .../contrib/udaf/example/UDAFExampleMax2.java | 268 ++++++++++ .../contrib/udf/example/GenericUDFTrim2.java | 46 ++ .../contrib/udf/example/UDFExampleAdd2.java | 38 ++ .../udtf/example/GenericUDTFCount3.java | 68 +++ .../hive/ql/udf/generic/GenericUDAFSum2.java | 462 ++++++++++++++++++ .../sql/hive/HiveUDFDynamicLoadSuite.scala | 5 +- 11 files changed, 1004 insertions(+), 2 deletions(-) create mode 100644 sql/hive/src/test/noclasspath/pom.xml create mode 100644 sql/hive/src/test/noclasspath/src/main/java/org/apache/hadoop/hive/contrib/udaf/example/UDAFExampleMax2.java create mode 100644 sql/hive/src/test/noclasspath/src/main/java/org/apache/hadoop/hive/contrib/udf/example/GenericUDFTrim2.java create mode 100644 sql/hive/src/test/noclasspath/src/main/java/org/apache/hadoop/hive/contrib/udf/example/UDFExampleAdd2.java create mode 100644 sql/hive/src/test/noclasspath/src/main/java/org/apache/hadoop/hive/contrib/udtf/example/GenericUDTFCount3.java create mode 100644 sql/hive/src/test/noclasspath/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFSum2.java diff --git a/pom.xml b/pom.xml index a7d750e81006..9c2ce0fe3fcd 100644 --- a/pom.xml +++ b/pom.xml @@ -335,6 +335,7 @@ -Djdk.reflect.useDirectMethodHandle=false -Dio.netty.tryReflectionSetAccessible=true --enable-native-access=ALL-UNNAMED + -Dmvn.executable=${maven.multiModuleProjectDirectory}/build/mvn 2.7.12 9.2.0 @@ -2667,6 +2668,12 @@ javax.servlet-api ${javaxservlet.version} + + org.apache.maven.shared + maven-invoker + 3.3.0 + test + diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 9406d8026a91..b0b94324f401 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -283,6 +283,10 @@ bcpkix-jdk18on test + + org.apache.maven.shared + maven-invoker + target/scala-${scala.binary.version}/classes diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala index f0f3f94b811f..c08d4417e442 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.test -import java.io.File +import java.io.{File, InputStream} import java.net.URI import java.nio.file.Files import java.util.{Locale, UUID} @@ -28,9 +28,11 @@ import scala.language.implicitConversions import scala.util.control.NonFatal import org.apache.hadoop.fs.Path +import org.apache.maven.shared.invoker.{DefaultInvocationRequest, DefaultInvoker} import org.scalactic.source.Position import org.scalatest.{BeforeAndAfterAll, Suite, Tag} import org.scalatest.concurrent.Eventually +import org.scalatest.matchers.should.Matchers._ import org.apache.spark.SparkFunSuite import org.apache.spark.sql.{AnalysisException, Row} @@ -535,4 +537,16 @@ private[sql] object SQLTestUtils { None } } + + private val invoker = new DefaultInvoker() + invoker.setMavenExecutable(new File(System.getProperty("mvn.executable", "mvn"))) + + final def invokeMaven(pomFile: File): Unit = { + val request = new DefaultInvocationRequest() + .setPomFile(pomFile) + .addArgs(Seq("clean", "package", "-DskipTests").toList.asJava) + .setInputStream(InputStream.nullInputStream()) + invoker.execute(request).getExitCode should equal (0) + } + } diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index c05699d63bfc..a5b8c8cc30a9 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -248,6 +248,10 @@ scalacheck_${scala.binary.version} test + + org.apache.maven.shared + maven-invoker + diff --git a/sql/hive/src/test/noclasspath/pom.xml b/sql/hive/src/test/noclasspath/pom.xml new file mode 100644 index 000000000000..65a7a5f39e55 --- /dev/null +++ b/sql/hive/src/test/noclasspath/pom.xml @@ -0,0 +1,88 @@ + + + + 4.0.0 + + org.apache.spark + hive-test-udfs + 4.1.0-SNAPSHOT + jar + Spark Hive Test UDFS + + + Apache-2.0 + https://www.apache.org/licenses/LICENSE-2.0.html + repo + + + + Apache Software Foundation + https://www.apache.org + + + + 8 + 8 + 3.4.1 + 2.3.10 + + + + ${project.artifactId} + + + maven-jar-plugin + + ${project.basedir} + + + + + + + + org.apache.hive + hive-exec + ${hive.version} + core + provided + + + org.apache.hive + hive-serde + ${hive.version} + provided + + + org.apache.hive + hive-storage-api + 2.4.0 + provided + + + org.apache.hadoop + hadoop-common + ${hadoop.version} + + + commons-logging + commons-logging + 1.1.3 + + + + diff --git a/sql/hive/src/test/noclasspath/src/main/java/org/apache/hadoop/hive/contrib/udaf/example/UDAFExampleMax2.java b/sql/hive/src/test/noclasspath/src/main/java/org/apache/hadoop/hive/contrib/udaf/example/UDAFExampleMax2.java new file mode 100644 index 000000000000..00681405dd3c --- /dev/null +++ b/sql/hive/src/test/noclasspath/src/main/java/org/apache/hadoop/hive/contrib/udaf/example/UDAFExampleMax2.java @@ -0,0 +1,268 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.contrib.udaf.example; + +import org.apache.hadoop.hive.ql.exec.Description; +import org.apache.hadoop.hive.ql.exec.UDAF; +import org.apache.hadoop.hive.ql.exec.UDAFEvaluator; +import org.apache.hadoop.hive.serde2.io.DoubleWritable; +import org.apache.hadoop.hive.serde2.io.ShortWritable; +import org.apache.hadoop.io.FloatWritable; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.Text; + +@Description(name = "udaf_max2", value = "_FUNC_(expr) - Returns the maximum value of expr") +public class UDAFExampleMax2 extends UDAF { + + static public class MaxShortEvaluator implements UDAFEvaluator { + private short mMax; + private boolean mEmpty; + + public MaxShortEvaluator() { + super(); + init(); + } + + public void init() { + mMax = 0; + mEmpty = true; + } + + public boolean iterate(ShortWritable o) { + if (o != null) { + if (mEmpty) { + mMax = o.get(); + mEmpty = false; + } else { + mMax = (short) Math.max(mMax, o.get()); + } + } + return true; + } + + public ShortWritable terminatePartial() { + return mEmpty ? null : new ShortWritable(mMax); + } + + public boolean merge(ShortWritable o) { + return iterate(o); + } + + public ShortWritable terminate() { + return mEmpty ? null : new ShortWritable(mMax); + } + } + + static public class MaxIntEvaluator implements UDAFEvaluator { + private int mMax; + private boolean mEmpty; + + public MaxIntEvaluator() { + super(); + init(); + } + + public void init() { + mMax = 0; + mEmpty = true; + } + + public boolean iterate(IntWritable o) { + if (o != null) { + if (mEmpty) { + mMax = o.get(); + mEmpty = false; + } else { + mMax = Math.max(mMax, o.get()); + } + } + return true; + } + + public IntWritable terminatePartial() { + return mEmpty ? null : new IntWritable(mMax); + } + + public boolean merge(IntWritable o) { + return iterate(o); + } + + public IntWritable terminate() { + return mEmpty ? null : new IntWritable(mMax); + } + } + + static public class MaxLongEvaluator implements UDAFEvaluator { + private long mMax; + private boolean mEmpty; + + public MaxLongEvaluator() { + super(); + init(); + } + + public void init() { + mMax = 0; + mEmpty = true; + } + + public boolean iterate(LongWritable o) { + if (o != null) { + if (mEmpty) { + mMax = o.get(); + mEmpty = false; + } else { + mMax = Math.max(mMax, o.get()); + } + } + return true; + } + + public LongWritable terminatePartial() { + return mEmpty ? null : new LongWritable(mMax); + } + + public boolean merge(LongWritable o) { + return iterate(o); + } + + public LongWritable terminate() { + return mEmpty ? null : new LongWritable(mMax); + } + } + + static public class MaxFloatEvaluator implements UDAFEvaluator { + private float mMax; + private boolean mEmpty; + + public MaxFloatEvaluator() { + super(); + init(); + } + + public void init() { + mMax = 0; + mEmpty = true; + } + + public boolean iterate(FloatWritable o) { + if (o != null) { + if (mEmpty) { + mMax = o.get(); + mEmpty = false; + } else { + mMax = Math.max(mMax, o.get()); + } + } + return true; + } + + public FloatWritable terminatePartial() { + return mEmpty ? null : new FloatWritable(mMax); + } + + public boolean merge(FloatWritable o) { + return iterate(o); + } + + public FloatWritable terminate() { + return mEmpty ? null : new FloatWritable(mMax); + } + } + + static public class MaxDoubleEvaluator implements UDAFEvaluator { + private double mMax; + private boolean mEmpty; + + public MaxDoubleEvaluator() { + super(); + init(); + } + + public void init() { + mMax = 0; + mEmpty = true; + } + + public boolean iterate(DoubleWritable o) { + if (o != null) { + if (mEmpty) { + mMax = o.get(); + mEmpty = false; + } else { + mMax = Math.max(mMax, o.get()); + } + } + return true; + } + + public DoubleWritable terminatePartial() { + return mEmpty ? null : new DoubleWritable(mMax); + } + + public boolean merge(DoubleWritable o) { + return iterate(o); + } + + public DoubleWritable terminate() { + return mEmpty ? null : new DoubleWritable(mMax); + } + } + + static public class MaxStringEvaluator implements UDAFEvaluator { + private Text mMax; + private boolean mEmpty; + + public MaxStringEvaluator() { + super(); + init(); + } + + public void init() { + mMax = null; + mEmpty = true; + } + + public boolean iterate(Text o) { + if (o != null) { + if (mEmpty) { + mMax = new Text(o); + mEmpty = false; + } else if (mMax.compareTo(o) < 0) { + mMax.set(o); + } + } + return true; + } + + public Text terminatePartial() { + return mEmpty ? null : mMax; + } + + public boolean merge(Text o) { + return iterate(o); + } + + public Text terminate() { + return mEmpty ? null : mMax; + } + } + +} \ No newline at end of file diff --git a/sql/hive/src/test/noclasspath/src/main/java/org/apache/hadoop/hive/contrib/udf/example/GenericUDFTrim2.java b/sql/hive/src/test/noclasspath/src/main/java/org/apache/hadoop/hive/contrib/udf/example/GenericUDFTrim2.java new file mode 100644 index 000000000000..4a6f05e350ac --- /dev/null +++ b/sql/hive/src/test/noclasspath/src/main/java/org/apache/hadoop/hive/contrib/udf/example/GenericUDFTrim2.java @@ -0,0 +1,46 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.contrib.udf.example; + +import org.apache.commons.lang.StringUtils; +import org.apache.hadoop.hive.ql.exec.Description; +import org.apache.hadoop.hive.ql.exec.vector.VectorizedExpressions; +import org.apache.hadoop.hive.ql.exec.vector.expressions.StringTrim; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBaseTrim; + +/** + * UDFTrim. + * + */ +@Description(name = "generic_udf_trim2", + value = "_FUNC_(str) - Removes the leading and trailing space characters from str ", + extended = "Example:\n" + + " > SELECT _FUNC_(' facebook ') FROM src LIMIT 1;\n" + " 'facebook'") +@VectorizedExpressions({ StringTrim.class }) +public class GenericUDFTrim2 extends GenericUDFBaseTrim { + public GenericUDFTrim2() { + super("generic_udf_trim2"); + } + + @Override + protected String performOp(String val) { + return StringUtils.strip(val, " "); + } + +} \ No newline at end of file diff --git a/sql/hive/src/test/noclasspath/src/main/java/org/apache/hadoop/hive/contrib/udf/example/UDFExampleAdd2.java b/sql/hive/src/test/noclasspath/src/main/java/org/apache/hadoop/hive/contrib/udf/example/UDFExampleAdd2.java new file mode 100644 index 000000000000..33e6f0954282 --- /dev/null +++ b/sql/hive/src/test/noclasspath/src/main/java/org/apache/hadoop/hive/contrib/udf/example/UDFExampleAdd2.java @@ -0,0 +1,38 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.contrib.udf.example; + +import org.apache.hadoop.hive.ql.exec.Description; +import org.apache.hadoop.hive.ql.exec.UDF; + +/** + * UDFExampleAdd2. + * + */ +@Description(name = "udf_add2", value = "_FUNC_(expr) - Example UDF that returns the sum") +public class UDFExampleAdd2 extends UDF { + + public Integer evaluate(Integer a, Integer b) { + return a + b; + } + + public Double evaluate(Double a, Double b) { + return a + b; + } + +} \ No newline at end of file diff --git a/sql/hive/src/test/noclasspath/src/main/java/org/apache/hadoop/hive/contrib/udtf/example/GenericUDTFCount3.java b/sql/hive/src/test/noclasspath/src/main/java/org/apache/hadoop/hive/contrib/udtf/example/GenericUDTFCount3.java new file mode 100644 index 000000000000..46866d49046a --- /dev/null +++ b/sql/hive/src/test/noclasspath/src/main/java/org/apache/hadoop/hive/contrib/udtf/example/GenericUDTFCount3.java @@ -0,0 +1,68 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.contrib.udtf.example; + +import java.util.ArrayList; + +import org.apache.hadoop.hive.ql.exec.Description; +import org.apache.hadoop.hive.ql.exec.UDFArgumentException; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDTF; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; + + +/** + * GenericUDTFCount3 outputs the number of rows seen, three times. It's output three times + * to test outputting of rows on close with lateral view. + * + */ +@Description(name = "udtf_count3", + value = "_FUNC_(col) - UDF outputs the number of rows seen, three times.") +public class GenericUDTFCount3 extends GenericUDTF { + + private transient Integer count = Integer.valueOf(0); + private transient Object forwardObj[] = new Object[1]; + + @Override + public void close() throws HiveException { + forwardObj[0] = count; + forward(forwardObj); + forward(forwardObj); + forward(forwardObj); + } + + @Override + public StructObjectInspector initialize(ObjectInspector[] argOIs) throws UDFArgumentException { + ArrayList fieldNames = new ArrayList(); + ArrayList fieldOIs = new ArrayList(); + fieldNames.add("col1"); + fieldOIs.add(PrimitiveObjectInspectorFactory.javaIntObjectInspector); + return ObjectInspectorFactory.getStandardStructObjectInspector(fieldNames, + fieldOIs); + } + + @Override + public void process(Object[] args) throws HiveException { + count = Integer.valueOf(count.intValue() + 1); + } + +} \ No newline at end of file diff --git a/sql/hive/src/test/noclasspath/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFSum2.java b/sql/hive/src/test/noclasspath/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFSum2.java new file mode 100644 index 000000000000..eee4029cbc17 --- /dev/null +++ b/sql/hive/src/test/noclasspath/src/main/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFSum2.java @@ -0,0 +1,462 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.ql.udf.generic; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.ql.exec.Description; +import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.parse.WindowingSpec.BoundarySpec; +import org.apache.hadoop.hive.ql.plan.ptf.BoundaryDef; +import org.apache.hadoop.hive.ql.plan.ptf.WindowFrameDef; +import org.apache.hadoop.hive.ql.util.JavaDataModel; +import org.apache.hadoop.hive.serde2.io.DoubleWritable; +import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils; +import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.util.StringUtils; + +/** + * GenericUDAFSum. + * + */ +@Description(name = "generic_udaf_sum2", value = "_FUNC_(x) - Returns the sum of a set of numbers") +public class GenericUDAFSum2 extends AbstractGenericUDAFResolver { + + static final Log LOG = LogFactory.getLog(GenericUDAFSum2.class.getName()); + + @Override + public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters) + throws SemanticException { + if (parameters.length != 1) { + throw new UDFArgumentTypeException(parameters.length - 1, + "Exactly one argument is expected."); + } + + if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE) { + throw new UDFArgumentTypeException(0, + "Only primitive type arguments are accepted but " + + parameters[0].getTypeName() + " is passed."); + } + switch (((PrimitiveTypeInfo) parameters[0]).getPrimitiveCategory()) { + case BYTE: + case SHORT: + case INT: + case LONG: + return new GenericUDAFSumLong(); + case TIMESTAMP: + case FLOAT: + case DOUBLE: + case STRING: + case VARCHAR: + case CHAR: + return new GenericUDAFSumDouble(); + case DECIMAL: + return new GenericUDAFSumHiveDecimal(); + case BOOLEAN: + case DATE: + default: + throw new UDFArgumentTypeException(0, + "Only numeric or string type arguments are accepted but " + + parameters[0].getTypeName() + " is passed."); + } + } + + public static PrimitiveObjectInspector.PrimitiveCategory getReturnType(TypeInfo type) { + if (type.getCategory() != ObjectInspector.Category.PRIMITIVE) { + return null; + } + switch (((PrimitiveTypeInfo) type).getPrimitiveCategory()) { + case BYTE: + case SHORT: + case INT: + case LONG: + return PrimitiveObjectInspector.PrimitiveCategory.LONG; + case TIMESTAMP: + case FLOAT: + case DOUBLE: + case STRING: + case VARCHAR: + case CHAR: + return PrimitiveObjectInspector.PrimitiveCategory.DOUBLE; + case DECIMAL: + return PrimitiveObjectInspector.PrimitiveCategory.DECIMAL; + } + return null; + } + + /** + * GenericUDAFSumHiveDecimal. + * + */ + public static class GenericUDAFSumHiveDecimal extends GenericUDAFEvaluator { + private PrimitiveObjectInspector inputOI; + private HiveDecimalWritable result; + + @Override + public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException { + assert (parameters.length == 1); + super.init(m, parameters); + result = new HiveDecimalWritable(HiveDecimal.ZERO); + inputOI = (PrimitiveObjectInspector) parameters[0]; + // The output precision is 10 greater than the input which should cover at least + // 10b rows. The scale is the same as the input. + DecimalTypeInfo outputTypeInfo = null; + if (mode == Mode.PARTIAL1 || mode == Mode.COMPLETE) { + int precision = Math.min(HiveDecimal.MAX_PRECISION, inputOI.precision() + 10); + outputTypeInfo = TypeInfoFactory.getDecimalTypeInfo(precision, inputOI.scale()); + } else { + outputTypeInfo = (DecimalTypeInfo) inputOI.getTypeInfo(); + } + return PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(outputTypeInfo); + } + + /** class for storing decimal sum value. */ + @AggregationType(estimable = false) // hard to know exactly for decimals + static class SumHiveDecimalAgg extends AbstractAggregationBuffer { + boolean empty; + HiveDecimal sum; + } + + @Override + public AggregationBuffer getNewAggregationBuffer() throws HiveException { + SumHiveDecimalAgg agg = new SumHiveDecimalAgg(); + reset(agg); + return agg; + } + + @Override + public void reset(AggregationBuffer agg) throws HiveException { + SumHiveDecimalAgg bdAgg = (SumHiveDecimalAgg) agg; + bdAgg.empty = true; + bdAgg.sum = HiveDecimal.ZERO; + } + + boolean warned = false; + + @Override + public void iterate(AggregationBuffer agg, Object[] parameters) throws HiveException { + assert (parameters.length == 1); + try { + merge(agg, parameters[0]); + } catch (NumberFormatException e) { + if (!warned) { + warned = true; + LOG.warn(getClass().getSimpleName() + " " + + StringUtils.stringifyException(e)); + LOG + .warn(getClass().getSimpleName() + + " ignoring similar exceptions."); + } + } + } + + @Override + public Object terminatePartial(AggregationBuffer agg) throws HiveException { + return terminate(agg); + } + + @Override + public void merge(AggregationBuffer agg, Object partial) throws HiveException { + if (partial != null) { + SumHiveDecimalAgg myagg = (SumHiveDecimalAgg) agg; + if (myagg.sum == null) { + return; + } + + myagg.empty = false; + myagg.sum = myagg.sum.add(PrimitiveObjectInspectorUtils.getHiveDecimal(partial, inputOI)); + } + } + + @Override + public Object terminate(AggregationBuffer agg) throws HiveException { + SumHiveDecimalAgg myagg = (SumHiveDecimalAgg) agg; + if (myagg.empty || myagg.sum == null) { + return null; + } + result.set(myagg.sum); + return result; + } + + @Override + public GenericUDAFEvaluator getWindowingEvaluator(WindowFrameDef wFrmDef) { + return new GenericUDAFStreamingEvaluator.SumAvgEnhancer( + this, wFrmDef) { + + @Override + protected HiveDecimalWritable getNextResult( + org.apache.hadoop.hive.ql.udf.generic.GenericUDAFStreamingEvaluator.SumAvgEnhancer.SumAvgStreamingState ss) + throws HiveException { + SumHiveDecimalAgg myagg = (SumHiveDecimalAgg) ss.wrappedBuf; + HiveDecimal r = myagg.empty ? null : myagg.sum; + HiveDecimal d = ss.retrieveNextIntermediateValue(); + + d = d == null ? HiveDecimal.ZERO : d; + r = r == null ? null : r.subtract(d); + + return r == null ? null : new HiveDecimalWritable(r); + } + + @Override + protected HiveDecimal getCurrentIntermediateResult( + org.apache.hadoop.hive.ql.udf.generic.GenericUDAFStreamingEvaluator.SumAvgEnhancer.SumAvgStreamingState ss) + throws HiveException { + SumHiveDecimalAgg myagg = (SumHiveDecimalAgg) ss.wrappedBuf; + return myagg.empty ? null : myagg.sum; + } + + }; + } + } + + /** + * GenericUDAFSumDouble. + * + */ + public static class GenericUDAFSumDouble extends GenericUDAFEvaluator { + private PrimitiveObjectInspector inputOI; + private DoubleWritable result; + + @Override + public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException { + assert (parameters.length == 1); + super.init(m, parameters); + result = new DoubleWritable(0); + inputOI = (PrimitiveObjectInspector) parameters[0]; + return PrimitiveObjectInspectorFactory.writableDoubleObjectInspector; + } + + /** class for storing double sum value. */ + @AggregationType(estimable = true) + static class SumDoubleAgg extends AbstractAggregationBuffer { + boolean empty; + double sum; + @Override + public int estimate() { return JavaDataModel.PRIMITIVES1 + JavaDataModel.PRIMITIVES2; } + } + + @Override + public AggregationBuffer getNewAggregationBuffer() throws HiveException { + SumDoubleAgg result = new SumDoubleAgg(); + reset(result); + return result; + } + + @Override + public void reset(AggregationBuffer agg) throws HiveException { + SumDoubleAgg myagg = (SumDoubleAgg) agg; + myagg.empty = true; + myagg.sum = 0; + } + + boolean warned = false; + + @Override + public void iterate(AggregationBuffer agg, Object[] parameters) throws HiveException { + assert (parameters.length == 1); + try { + merge(agg, parameters[0]); + } catch (NumberFormatException e) { + if (!warned) { + warned = true; + LOG.warn(getClass().getSimpleName() + " " + + StringUtils.stringifyException(e)); + LOG + .warn(getClass().getSimpleName() + + " ignoring similar exceptions."); + } + } + } + + @Override + public Object terminatePartial(AggregationBuffer agg) throws HiveException { + return terminate(agg); + } + + @Override + public void merge(AggregationBuffer agg, Object partial) throws HiveException { + if (partial != null) { + SumDoubleAgg myagg = (SumDoubleAgg) agg; + myagg.empty = false; + myagg.sum += PrimitiveObjectInspectorUtils.getDouble(partial, inputOI); + } + } + + @Override + public Object terminate(AggregationBuffer agg) throws HiveException { + SumDoubleAgg myagg = (SumDoubleAgg) agg; + if (myagg.empty) { + return null; + } + result.set(myagg.sum); + return result; + } + + @Override + public GenericUDAFEvaluator getWindowingEvaluator(WindowFrameDef wFrmDef) { + return new GenericUDAFStreamingEvaluator.SumAvgEnhancer(this, + wFrmDef) { + + @Override + protected DoubleWritable getNextResult( + org.apache.hadoop.hive.ql.udf.generic.GenericUDAFStreamingEvaluator.SumAvgEnhancer.SumAvgStreamingState ss) + throws HiveException { + SumDoubleAgg myagg = (SumDoubleAgg) ss.wrappedBuf; + Double r = myagg.empty ? null : myagg.sum; + Double d = ss.retrieveNextIntermediateValue(); + d = d == null ? (double)0.0F : d; + r = r == null ? null : r - d; + + return r == null ? null : new DoubleWritable(r); + } + + @Override + protected Double getCurrentIntermediateResult( + org.apache.hadoop.hive.ql.udf.generic.GenericUDAFStreamingEvaluator.SumAvgEnhancer.SumAvgStreamingState ss) + throws HiveException { + SumDoubleAgg myagg = (SumDoubleAgg) ss.wrappedBuf; + return myagg.empty ? null : new Double(myagg.sum); + } + + }; + } + + } + + /** + * GenericUDAFSumLong. + * + */ + public static class GenericUDAFSumLong extends GenericUDAFEvaluator { + private PrimitiveObjectInspector inputOI; + private LongWritable result; + + @Override + public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException { + assert (parameters.length == 1); + super.init(m, parameters); + result = new LongWritable(0); + inputOI = (PrimitiveObjectInspector) parameters[0]; + return PrimitiveObjectInspectorFactory.writableLongObjectInspector; + } + + /** class for storing double sum value. */ + @AggregationType(estimable = true) + static class SumLongAgg extends AbstractAggregationBuffer { + boolean empty; + long sum; + @Override + public int estimate() { return JavaDataModel.PRIMITIVES1 + JavaDataModel.PRIMITIVES2; } + } + + @Override + public AggregationBuffer getNewAggregationBuffer() throws HiveException { + SumLongAgg result = new SumLongAgg(); + reset(result); + return result; + } + + @Override + public void reset(AggregationBuffer agg) throws HiveException { + SumLongAgg myagg = (SumLongAgg) agg; + myagg.empty = true; + myagg.sum = 0; + } + + private boolean warned = false; + + @Override + public void iterate(AggregationBuffer agg, Object[] parameters) throws HiveException { + assert (parameters.length == 1); + try { + merge(agg, parameters[0]); + } catch (NumberFormatException e) { + if (!warned) { + warned = true; + LOG.warn(getClass().getSimpleName() + " " + + StringUtils.stringifyException(e)); + } + } + } + + @Override + public Object terminatePartial(AggregationBuffer agg) throws HiveException { + return terminate(agg); + } + + @Override + public void merge(AggregationBuffer agg, Object partial) throws HiveException { + if (partial != null) { + SumLongAgg myagg = (SumLongAgg) agg; + myagg.sum += PrimitiveObjectInspectorUtils.getLong(partial, inputOI); + myagg.empty = false; + } + } + + @Override + public Object terminate(AggregationBuffer agg) throws HiveException { + SumLongAgg myagg = (SumLongAgg) agg; + if (myagg.empty) { + return null; + } + result.set(myagg.sum); + return result; + } + + @Override + public GenericUDAFEvaluator getWindowingEvaluator(WindowFrameDef wFrmDef) { + return new GenericUDAFStreamingEvaluator.SumAvgEnhancer(this, + wFrmDef) { + + @Override + protected LongWritable getNextResult( + org.apache.hadoop.hive.ql.udf.generic.GenericUDAFStreamingEvaluator.SumAvgEnhancer.SumAvgStreamingState ss) + throws HiveException { + SumLongAgg myagg = (SumLongAgg) ss.wrappedBuf; + Long r = myagg.empty ? null : myagg.sum; + Long d = ss.retrieveNextIntermediateValue(); + d = d == null ? 0L : d; + r = r == null ? null : r - d; + + return r == null ? null : new LongWritable(r); + } + + @Override + protected Long getCurrentIntermediateResult( + org.apache.hadoop.hive.ql.udf.generic.GenericUDAFStreamingEvaluator.SumAvgEnhancer.SumAvgStreamingState ss) + throws HiveException { + SumLongAgg myagg = (SumLongAgg) ss.wrappedBuf; + return myagg.empty ? null : new Long(myagg.sum); + } + + }; + } + } + +} \ No newline at end of file diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUDFDynamicLoadSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUDFDynamicLoadSuite.scala index 4b27082e188f..68ccf049a915 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUDFDynamicLoadSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUDFDynamicLoadSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.hive +import java.io.File + import org.apache.spark.sql.{QueryTest, Row} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression} import org.apache.spark.sql.hive.HiveShim.HiveFunctionWrapper @@ -140,6 +142,8 @@ class HiveUDFDynamicLoadSuite extends QueryTest with SQLTestUtils with TestHiveS ) ).toImmutableArraySeq + SQLTestUtils.invokeMaven(new File("src/test/noclasspath/pom.xml")) + udfTestInfos.foreach { udfInfo => // The test jars are built from below commit: // https://github.com/HeartSaVioR/hive/commit/12f3f036b6efd0299cd1d457c0c0a65e0fd7e5f2 @@ -147,7 +151,6 @@ class HiveUDFDynamicLoadSuite extends QueryTest with SQLTestUtils with TestHiveS // This jar file should not be placed to the classpath. val jarPath = "src/test/noclasspath/hive-test-udfs.jar" - assume(new java.io.File(jarPath).exists) val jarUrl = s"file://${System.getProperty("user.dir")}/$jarPath" test("Spark should be able to run Hive UDF using jar regardless of " + From 6b396b2cbe1346445edf0a50d64be3fc6c18d3ee Mon Sep 17 00:00:00 2001 From: Vlad Rozov Date: Mon, 5 May 2025 09:31:02 -0700 Subject: [PATCH 06/19] remove from test-jars.txt --- dev/test-jars.txt | 1 - 1 file changed, 1 deletion(-) diff --git a/dev/test-jars.txt b/dev/test-jars.txt index bd8fc93bc9f0..58e2bbef0b8d 100644 --- a/dev/test-jars.txt +++ b/dev/test-jars.txt @@ -10,7 +10,6 @@ sql/connect/common/src/test/resources/artifact-tests/smallJar.jar sql/core/src/test/resources/SPARK-33084.jar sql/core/src/test/resources/artifact-tests/udf_noA.jar sql/hive-thriftserver/src/test/resources/TestUDTF.jar -sql/hive/src/test/noclasspath/hive-test-udfs.jar sql/hive/src/test/resources/SPARK-21101-1.0.jar sql/hive/src/test/resources/TestUDTF.jar sql/hive/src/test/resources/data/files/TestSerDe.jar From 50e67eec18244ac13a0b8c45b00159ccf5436d7b Mon Sep 17 00:00:00 2001 From: Vlad Rozov Date: Tue, 20 May 2025 14:03:36 +0000 Subject: [PATCH 07/19] [WIP][SPARK-51348][BUILD][SQL] 15 tests fails --- assembly/pom.xml | 28 ++++++++++++ .../spark/deploy/worker/ExecutorRunner.scala | 15 ++++--- .../spark/deploy/SparkSubmitTestUtils.scala | 1 + pom.xml | 29 +++++++++++- project/SparkBuild.scala | 1 + .../execution/datasources/DaysWritable.scala | 15 ++++--- .../datasources/orc/OrcSerializer.scala | 2 +- .../datasources/orc/OrcShimUtils.scala | 12 ++--- .../sql/execution/command/DDLSuite.scala | 3 +- sql/hive-thriftserver/pom.xml | 4 ++ .../apache/hive/service/cli/RowBasedSet.java | 14 +++--- .../hive/thriftserver/SparkSQLCLIDriver.scala | 42 ++++++++++-------- .../server/SparkSQLOperationManager.scala | 5 +-- .../sql/hive/thriftserver/CliSuite.scala | 6 ++- .../HiveCliSessionStateSuite.scala | 2 + .../SparkMetadataOperationSuite.scala | 35 +++++++-------- ...arkThriftServerProtocolVersionsSuite.scala | 4 +- .../execution/HiveCompatibilitySuite.scala | 4 +- sql/hive/pom.xml | 12 ++--- .../spark/sql/hive/HiveInspectors.scala | 4 +- .../sql/hive/client/HiveClientImpl.scala | 23 ++++++++-- .../HiveScriptTransformationExec.scala | 14 +++--- sql/hive/src/test/noclasspath/pom.xml | 24 +++------- .../contrib/udf/example/GenericUDFTrim2.java | 9 ++-- sql/hive/src/test/resources/TestUDTF.jar | Bin 0 -> 1328 bytes .../sql/hive/ClasspathDependenciesSuite.scala | 2 +- .../sql/hive/HiveMetastoreCatalogSuite.scala | 2 +- .../spark/sql/hive/HiveSparkSubmitSuite.scala | 10 ++--- .../spark/sql/hive/HiveUtilsSuite.scala | 8 ++-- .../hive/client/HadoopVersionInfoSuite.scala | 4 +- .../sql/hive/client/HiveVersionSuite.scala | 2 +- .../sql/hive/execution/HiveDDLSuite.scala | 6 ++- .../sql/hive/execution/HiveUDFSuite.scala | 3 +- .../sql/hive/orc/HiveOrcSourceSuite.scala | 21 ++++++--- 34 files changed, 231 insertions(+), 135 deletions(-) create mode 100644 sql/hive/src/test/resources/TestUDTF.jar diff --git a/assembly/pom.xml b/assembly/pom.xml index f5e9bc20f37c..350168d57f94 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -264,12 +264,40 @@ hive-thriftserver + + provided + org.apache.spark spark-hive-thriftserver_${scala.binary.version} ${project.version} + + org.eclipse.jetty + jetty-server + runtime + + + org.eclipse.jetty + jetty-security + runtime + + + org.eclipse.jetty + jetty-servlet + runtime + + + org.apache.datasketches + datasketches-hive + runtime + + + ${hive.group} + hive-llap-common + runtime + diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 8f0b684a93e8..75d3c7caa660 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -80,13 +80,16 @@ private[deploy] class ExecutorRunner( } workerThread.start() // Shutdown hook that kills actors on shutdown. - shutdownHook = ShutdownHookManager.addShutdownHook { () => - // It's possible that we arrive here before calling `fetchAndRunExecutor`, then `state` will - // be `ExecutorState.LAUNCHING`. In this case, we should set `state` to `FAILED`. - if (state == ExecutorState.LAUNCHING || state == ExecutorState.RUNNING) { - state = ExecutorState.FAILED + if (state == ExecutorState.LAUNCHING || state == ExecutorState.RUNNING) { + shutdownHook = ShutdownHookManager.addShutdownHook { () => + // It's possible that we arrive here before calling `fetchAndRunExecutor`, then `state` will + // be `ExecutorState.LAUNCHING`. In this case, we should set `state` to `FAILED`. + if (state == ExecutorState.LAUNCHING || state == ExecutorState.RUNNING) { + state = ExecutorState.FAILED + } + killProcess("Worker shutting down") } - killProcess("Worker shutting down") } + } } /** diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitTestUtils.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitTestUtils.scala index e38be4cefc07..b9103d74bbb9 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitTestUtils.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitTestUtils.scala @@ -59,6 +59,7 @@ trait SparkSubmitTestUtils extends SparkFunSuite with TimeLimits { val env = builder.environment() if (isSparkTesting) { env.put("SPARK_TESTING", "1") + // env.put("DEFAULT_ARTIFACT_REPOSITORY", "file:///local/home/vrozov/.m2/repository/") } else { env.remove("SPARK_TESTING") env.remove("SPARK_SQL_TESTING") diff --git a/pom.xml b/pom.xml index 9c2ce0fe3fcd..e5464717b419 100644 --- a/pom.xml +++ b/pom.xml @@ -1653,6 +1653,10 @@ ${hive.group} hive-metastore + + ${hive.group} + hive-standalone-metastore-server + ${hive.group} hive-service @@ -2193,7 +2197,7 @@ ${hive.group} hive-service-rpc - 4.0.0 + ${hive.version} * @@ -2348,6 +2352,29 @@ org.antlr antlr-runtime + + org.apache.orc + orc-core + + + org.apache.datasketches + datasketches-hive + + + com.github.joshelser + dropwizard-metrics-hadoop-metrics2-reporter + + + + + org.apache.datasketches + datasketches-hive + 2.0.0 + + + org.slf4j + slf4j-simple + diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index d99fa67c1212..9be58ea49dbe 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -1750,6 +1750,7 @@ object TestSettings { }.getOrElse(Nil): _*), // Show full stack trace and duration in test cases. (Test / testOptions) += Tests.Argument("-oDF"), + (Test / testOptions) += Tests.Argument(TestFrameworks.ScalaTest, "-fG", "scalatest.txt"), // Slowpoke notifications: receive notifications every 5 minute of tests that have been running // longer than two minutes. (Test / testOptions) += Tests.Argument(TestFrameworks.ScalaTest, "-W", "120", "300"), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DaysWritable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DaysWritable.scala index 93be706c7868..a04c2fcbbac1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DaysWritable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DaysWritable.scala @@ -18,9 +18,9 @@ package org.apache.spark.sql.execution.datasources import java.io.{DataInput, DataOutput, IOException} +import java.sql.Date -import org.apache.hadoop.hive.common.`type`.Date -import org.apache.hadoop.hive.serde2.io.DateWritableV2 +import org.apache.hadoop.hive.serde2.io.DateWritable import org.apache.hadoop.io.WritableUtils import org.apache.spark.sql.catalyst.util.RebaseDateTime.{rebaseGregorianToJulianDays, rebaseJulianToGregorianDays} @@ -38,16 +38,16 @@ import org.apache.spark.sql.catalyst.util.RebaseDateTime.{rebaseGregorianToJulia class DaysWritable( var gregorianDays: Int, var julianDays: Int) - extends DateWritableV2 { + extends DateWritable { def this() = this(0, 0) def this(gregorianDays: Int) = this(gregorianDays, rebaseGregorianToJulianDays(gregorianDays)) - def this(dateWritable: DateWritableV2) = { + def this(dateWritable: DateWritable) = { this( gregorianDays = dateWritable match { case daysWritable: DaysWritable => daysWritable.gregorianDays - case dateWritable: DateWritableV2 => + case dateWritable: DateWritable => rebaseJulianToGregorianDays(dateWritable.getDays) }, julianDays = dateWritable.getDays) @@ -55,7 +55,10 @@ class DaysWritable( override def getDays: Int = julianDays override def get: Date = { - Date.ofEpochMilli(DateWritableV2.daysToMillis(julianDays)) + new Date(DateWritable.daysToMillis(julianDays)) + } + override def get(doesTimeMatter: Boolean): Date = { + new Date(DateWritable.daysToMillis(julianDays, doesTimeMatter)) } override def set(d: Int): Unit = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcSerializer.scala index 33bad3b45c0f..8b74405b74f7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcSerializer.scala @@ -137,7 +137,7 @@ class OrcSerializer(dataSchema: StructType) { new BytesWritable(getter.getBinary(ordinal)) case DateType => - OrcShimUtils.getDateWritableV2(reuseObj) + OrcShimUtils.getDateWritable(reuseObj) // The following cases are already expensive, reusing object or not doesn't matter. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala index 04c023cb978f..60c5b7a266c5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala @@ -21,10 +21,10 @@ import org.apache.hadoop.hive.common.`type`.HiveDecimal import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch import org.apache.hadoop.hive.ql.io.sarg.{SearchArgument => OrcSearchArgument} import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf.{Operator => OrcOperator} -import org.apache.hadoop.hive.serde2.io.{DateWritableV2, HiveDecimalWritable} +import org.apache.hadoop.hive.serde2.io.{DateWritable, HiveDecimalWritable} import org.apache.spark.sql.catalyst.expressions.SpecializedGetters -import org.apache.spark.sql.catalyst.util.HiveDateTimeUtils +import org.apache.spark.sql.execution.datasources.DaysWritable import org.apache.spark.sql.types.Decimal /** @@ -38,7 +38,7 @@ private[sql] object OrcShimUtils { private[sql] type SearchArgument = OrcSearchArgument def getGregorianDays(value: Any): Int = { - HiveDateTimeUtils.fromHiveDate(value.asInstanceOf[DateWritableV2].get) + new DaysWritable(value.asInstanceOf[DateWritable]).gregorianDays } def getDecimal(value: Any): Decimal = { @@ -46,15 +46,15 @@ private[sql] object OrcShimUtils { Decimal(decimal.bigDecimalValue, decimal.precision(), decimal.scale()) } - def getDateWritableV2(reuseObj: Boolean): (SpecializedGetters, Int) => DateWritableV2 = { + def getDateWritable(reuseObj: Boolean): (SpecializedGetters, Int) => DateWritable = { if (reuseObj) { - val result = new DateWritableV2() + val result = new DaysWritable() (getter, ordinal) => result.set(getter.getInt(ordinal)) result } else { (getter: SpecializedGetters, ordinal: Int) => - new DateWritableV2(HiveDateTimeUtils.toHiveDate(getter.getInt(ordinal))) + new DaysWritable(getter.getInt(ordinal)) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 3d4da878869e..87c5b38a8184 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -2116,7 +2116,8 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { } } - test(s"Support alter table command with CASE_SENSITIVE is true") { + test("Support alter table command with CASE_SENSITIVE is true") { + assume(isInstanceOf[InMemoryCatalogedDDLSuite], "Hive is no longer case preserving") withSQLConf(SQLConf.CASE_SENSITIVE.key -> s"true") { withLocale("tr") { val dbName = "DaTaBaSe_I" diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 135b84cd01f8..7741e04547a8 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -81,6 +81,10 @@ ${hive.group} hive-beeline + + ${hive.group} + hive-standalone-metastore-server + ${hive.group} hive-service-rpc diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowBasedSet.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowBasedSet.java index 2452691a2ffe..3276d83594d8 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowBasedSet.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowBasedSet.java @@ -33,22 +33,20 @@ public class RowBasedSet implements RowSet { private long startOffset; private final TypeDescriptor[] descriptors; // non-null only for writing (server-side) - private final RemovableList rows; + private final RemovableList rows; public RowBasedSet(TableSchema schema) { descriptors = schema.toTypeDescriptors(); - rows = new RemovableList(); + rows = new RemovableList(); } public RowBasedSet(TRowSet tRowSet) { - descriptors = null; - rows = new RemovableList(tRowSet.getRows()); - startOffset = tRowSet.getStartRowOffset(); + this(null, tRowSet.getRows(), tRowSet.getStartRowOffset()); } private RowBasedSet(TypeDescriptor[] descriptors, List rows, long startOffset) { this.descriptors = descriptors; - this.rows = new RemovableList(rows); + this.rows = rows != null ? new RemovableList(rows) : new RemovableList(); this.startOffset = startOffset; } @@ -128,9 +126,9 @@ public void remove() { }; } - private static class RemovableList extends ArrayList { + private static class RemovableList extends ArrayList { RemovableList() { super(); } - RemovableList(List rows) { super(rows); } + RemovableList(List rows) { super(rows); } @Override public void removeRange(int fromIndex, int toIndex) { super.removeRange(fromIndex, toIndex); diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index e1dd2560fa2d..96e504222479 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -33,6 +33,8 @@ import org.apache.hadoop.hive.common.HiveInterruptUtils import org.apache.hadoop.hive.common.io.SessionStream import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.Driver +import org.apache.hadoop.hive.ql.exec.mr.HadoopJobExecHelper +import org.apache.hadoop.hive.ql.exec.tez.TezJobExecHelper import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.security.{Credentials, UserGroupInformation} @@ -47,7 +49,6 @@ import org.apache.spark.sql.catalyst.analysis.FunctionRegistry import org.apache.spark.sql.catalyst.util.SQLKeywordUtils import org.apache.spark.sql.hive.client.HiveClientImpl import org.apache.spark.sql.hive.security.HiveDelegationTokenProvider -import org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.closeHiveSessionStateIfStarted import org.apache.spark.sql.internal.{SharedState, SQLConf} import org.apache.spark.sql.internal.SQLConf.LEGACY_EMPTY_CURRENT_DB_IN_CLI import org.apache.spark.util.{SparkStringUtils, Utils} @@ -105,12 +106,12 @@ private[hive] object SparkSQLCLIDriver extends Logging { sessionState.err = new SessionStream(System.err, true, UTF_8.name()) } catch { case e: UnsupportedEncodingException => - closeHiveSessionStateIfStarted(sessionState) + sessionState.close() exit(ERROR_PATH_NOT_FOUND) } if (!oproc.process_stage2(sessionState)) { - closeHiveSessionStateIfStarted(sessionState) + sessionState.close() exit(ERROR_MISUSE_SHELL_BUILTIN) } @@ -145,7 +146,7 @@ private[hive] object SparkSQLCLIDriver extends Logging { // Clean up after we exit ShutdownHookManager.addShutdownHook { () => - closeHiveSessionStateIfStarted(sessionState) + sessionState.close() SparkSQLEnv.stop(exitCode) } @@ -220,6 +221,7 @@ private[hive] object SparkSQLCLIDriver extends Logging { try { if (sessionState.fileName != null) { cli.processFile(sessionState.fileName) + exit(0) } } catch { case e: FileNotFoundException => @@ -298,8 +300,7 @@ private[hive] object SparkSQLCLIDriver extends Logging { try { cli.processLine(line, true) } catch { - case e: CommandProcessorException => - exit(e.getErrorCode) + case _: CommandProcessorException | _: RuntimeException => None } prefix = "" currentPrompt = promptWithCurrentDB @@ -311,7 +312,7 @@ private[hive] object SparkSQLCLIDriver extends Logging { line = reader.readLine(currentPrompt + "> ") } - closeHiveSessionStateIfStarted(sessionState) + sessionState.close() exit(0) } @@ -444,7 +445,7 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { val cmd_1: String = cmd_trimmed.substring(tokens(0).length()).trim() if (cmd_lower.equals("quit") || cmd_lower.equals("exit")) { - closeHiveSessionStateIfStarted(sessionState) + sessionState.close() SparkSQLCLIDriver.exit(EXIT_SUCCESS) } if (tokens(0).toLowerCase(Locale.ROOT).equals("source") || cmd_trimmed.startsWith("!")) { @@ -520,10 +521,10 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { |${Utils.stringifyException(e)} """.stripMargin) throw new CommandProcessorException(e) + } finally { + driver.close() } - driver.close() - var responseMsg = s"Time taken: $timeTaken seconds" if (counter != 0) { responseMsg += s", Fetched $counter row(s)" @@ -544,6 +545,7 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { } // Adapted processLine from Hive 2.3's CliDriver.processLine. + // Updated for Hive 4.1 override def processLine(line: String, allowInterrupting: Boolean): CommandProcessorResponse = { var oldSignal: SignalHandler = null var interruptSignal: Signal = null @@ -569,35 +571,37 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { // to prompt console.printInfo("Interrupting... Be patient, this might take some time.") console.printInfo("Press Ctrl+C again to kill JVM") - + // First, kill any running MR jobs + HadoopJobExecHelper.killRunningJobs() + TezJobExecHelper.killRunningJobs() HiveInterruptUtils.interrupt() } }) } try { - val ignoreErrors = - HiveConf.getBoolVar(conf, HiveConf.getConfVars("hive.cli.errors.ignore")) - var ret: CommandProcessorResponse = null - + var ret: CommandProcessorResponse = new CommandProcessorResponse // we can not use "split" function directly as ";" may be quoted val commands = splitSemiColon(line).asScala - var command: String = "" + val commandBuilder = new StringBuilder for (oneCmd <- commands) { if (oneCmd.endsWith("\\")) { - command += oneCmd.dropRight(1) + ";" + commandBuilder.append(oneCmd.dropRight(1) + ";") } else { - command += oneCmd + commandBuilder.append(oneCmd) + val command = commandBuilder.toString if (!SparkStringUtils.isBlank(command)) { try { ret = processCmd(command) } catch { case e: CommandProcessorException => + val ignoreErrors = HiveConf.getBoolVar(conf, HiveConf.ConfVars.CLI_IGNORE_ERRORS) if (!ignoreErrors) { throw e } + } finally { + commandBuilder.clear() } - command = "" } } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala index 5c81400289d2..da4a9299a0b8 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala @@ -127,16 +127,15 @@ private[thriftserver] class SparkSQLOperationManager() operation } - def newGetFunctionsOperation( + override def newGetFunctionsOperation( parentSession: HiveSession, - operationManager: OperationManager, catalogName: String, schemaName: String, functionName: String): GetFunctionsOperation = synchronized { val session = sessionToContexts.get(parentSession.getSessionHandle) require(session != null, s"Session handle: ${parentSession.getSessionHandle} has not been" + " initialized or had already closed.") - val operation = new SparkGetFunctionsOperation(session, parentSession, operationManager, + val operation = new SparkGetFunctionsOperation(session, parentSession, this, catalogName, schemaName, functionName) handleToOperation.put(operation.getHandle, operation) logDebug(s"Created GetFunctionsOperation with session=$parentSession.") diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index 655c3526f5cb..b2abb28a7e77 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.hive.thriftserver import java.io._ import java.nio.charset.StandardCharsets -import java.util.concurrent.CountDownLatch +import java.util.concurrent.{CountDownLatch, TimeUnit} import scala.collection.mutable.ArrayBuffer import scala.concurrent.Promise @@ -238,6 +238,7 @@ class CliSuite extends SparkFunSuite { // --hiveconf will overrides hive-site.xml runCliWithin(2.minute, useExternalHiveFile = true)( "desc database default;" -> warehousePath.getAbsolutePath, + "drop database if exists cliTestDb;" -> "", "create database cliTestDb;" -> "", "desc database cliTestDb;" -> warehousePath.getAbsolutePath, "set spark.sql.warehouse.dir;" -> warehousePath.getAbsolutePath) @@ -255,6 +256,7 @@ class CliSuite extends SparkFunSuite { useExternalHiveFile = true, metastore = metastore)( "desc database default;" -> sparkWareHouseDir.getAbsolutePath, + "drop database if exists cliTestDb;" -> "", "create database cliTestDb;" -> "", "desc database cliTestDb;" -> sparkWareHouseDir.getAbsolutePath, "set spark.sql.warehouse.dir;" -> sparkWareHouseDir.getAbsolutePath) @@ -697,7 +699,7 @@ class CliSuite extends SparkFunSuite { } t.start() t.start() - cd.await() + assert(cd.await(5, TimeUnit.MINUTES)) } // scalastyle:off line.size.limit diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveCliSessionStateSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveCliSessionStateSuite.scala index 035b71a37a69..423e9ed04510 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveCliSessionStateSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveCliSessionStateSuite.scala @@ -37,6 +37,8 @@ class HiveCliSessionStateSuite extends SparkFunSuite { HiveUtils.newTemporaryConfiguration(useInMemoryDerby = false).foreach { case (key, value) => hiveConf.set(key, value) } + // Set Hive execution engine to map-reduce for backward compatibility with Hive 2.3.x + HiveConf.setVar(hiveConf, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE, "mr") val sessionState: SessionState = new CliSessionState(hiveConf) SessionState.start(sessionState) val s1 = SessionState.get diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala index abd2b1983b34..41cfe928fbb7 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala @@ -471,24 +471,9 @@ class SparkMetadataOperationSuite extends HiveThriftServer2TestBase { val metaData = statement.getConnection.getMetaData Seq( () => metaData.allProceduresAreCallable, - () => metaData.getURL, - () => metaData.getUserName, () => metaData.isReadOnly, - () => metaData.nullsAreSortedHigh, - () => metaData.nullsAreSortedLow, - () => metaData.nullsAreSortedAtStart, - () => metaData.nullsAreSortedAtEnd, () => metaData.usesLocalFiles, () => metaData.usesLocalFilePerTable, - () => metaData.supportsMixedCaseIdentifiers, - () => metaData.supportsMixedCaseQuotedIdentifiers, - () => metaData.storesUpperCaseIdentifiers, - () => metaData.storesUpperCaseQuotedIdentifiers, - () => metaData.storesLowerCaseIdentifiers, - () => metaData.storesLowerCaseQuotedIdentifiers, - () => metaData.storesMixedCaseIdentifiers, - () => metaData.storesMixedCaseQuotedIdentifiers, - () => metaData.getSQLKeywords, () => metaData.nullPlusNonNullIsNull, () => metaData.supportsConvert, () => metaData.supportsTableCorrelationNames, @@ -569,7 +554,6 @@ class SparkMetadataOperationSuite extends HiveThriftServer2TestBase { () => metaData.getRowIdLifetime, () => metaData.supportsStoredFunctionsUsingCallSyntax, () => metaData.autoCommitFailureClosesAllResultSets, - () => metaData.getClientInfoProperties, () => metaData.getFunctionColumns("", "%", "%", "%"), () => metaData.getPseudoColumns("", "%", "%", "%"), () => metaData.generatedKeyAlwaysReturned).foreach { func => @@ -594,8 +578,7 @@ class SparkMetadataOperationSuite extends HiveThriftServer2TestBase { assert(metaData.getDriverVersion === HiveVersionInfo.getVersion) assert(metaData.getDatabaseMajorVersion === VersionUtils.majorVersion(SPARK_VERSION)) assert(metaData.getDatabaseMinorVersion === VersionUtils.minorVersion(SPARK_VERSION)) - assert(metaData.getIdentifierQuoteString === " ", - "This method returns a space \" \" if identifier quoting is not supported") + assert(metaData.getIdentifierQuoteString === "`") assert(metaData.getNumericFunctions === "") assert(metaData.getStringFunctions === "") assert(metaData.getSystemFunctions === "") @@ -658,6 +641,22 @@ class SparkMetadataOperationSuite extends HiveThriftServer2TestBase { assert(metaData.getSQLStateType === DatabaseMetaData.sqlStateSQL) assert(metaData.getMaxLogicalLobSize === 0) assert(!metaData.supportsRefCursors) + assert(metaData.getURL.startsWith("jdbc:hive2://")) + assert(metaData.getUserName === System.getProperty("user.name")) + assert(metaData.nullsAreSortedHigh) + assert(!metaData.nullsAreSortedLow) + assert(!metaData.nullsAreSortedAtStart) + assert(!metaData.nullsAreSortedAtEnd) + assert(!metaData.supportsMixedCaseIdentifiers) + assert(!metaData.supportsMixedCaseQuotedIdentifiers) + assert(!metaData.storesUpperCaseIdentifiers) + assert(!metaData.storesUpperCaseQuotedIdentifiers) + assert(metaData.storesLowerCaseIdentifiers) + assert(metaData.storesLowerCaseQuotedIdentifiers) + assert(!metaData.storesMixedCaseIdentifiers) + assert(!metaData.storesMixedCaseQuotedIdentifiers) + assert(metaData.getSQLKeywords != "") + assert(metaData.getClientInfoProperties != null) } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkThriftServerProtocolVersionsSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkThriftServerProtocolVersionsSuite.scala index 8b0093edfaf6..3581bfc478b1 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkThriftServerProtocolVersionsSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkThriftServerProtocolVersionsSuite.scala @@ -70,7 +70,9 @@ class SparkThriftServerProtocolVersionsSuite extends HiveThriftServer2TestBase { .build() f(rs) } finally { - rs.close() + if (rs != null) { + rs.close() + } connection.close() transport.close() rawTransport.close() diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 4f5f50433ea1..1ccc5cd4efb3 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -218,6 +218,9 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "alter1", "input16", + // Hive 4 does not support DynamicSerDe + "input19", + // No support for unpublished test udfs. "autogen_colalias", @@ -748,7 +751,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "input12_hadoop20", "input14", "input15", - "input19", "input1_limit", "input2", "input21", diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index a5b8c8cc30a9..72024f49a171 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -147,6 +147,11 @@ hive-standalone-metastore-server test + + org.apache.datasketches + datasketches-hive + test + ${hive.group} hive-parser @@ -159,12 +164,7 @@ org.apache.hadoop hadoop-hdfs ${hadoop.version} - test - - - org.apache.datasketches - datasketches-java - 3.1.0 + runtime diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index 1e2969eb4297..14993ae31824 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -329,10 +329,10 @@ private[hive] trait HiveInspectors { HiveDecimal.create(o.asInstanceOf[Decimal].toJavaBigDecimal)) case _: JavaDateObjectInspector => withNullSafe(o => - DateTimeUtils.toJavaDate(o.asInstanceOf[Int])) + HiveDateTimeUtils.toHiveDate(o.asInstanceOf[Int])) case _: JavaTimestampObjectInspector => withNullSafe(o => - DateTimeUtils.toJavaTimestamp(o.asInstanceOf[Long])) + HiveDateTimeUtils.toHiveTimestamp(o.asInstanceOf[Long])) case _: HiveDecimalObjectInspector if x.preferWritable() => withNullSafe(o => getDecimalWritable(o.asInstanceOf[Decimal])) case _: HiveDecimalObjectInspector => diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index 43d38b29901c..206edc2a59e5 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -28,6 +28,7 @@ import scala.annotation.tailrec import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.jdk.CollectionConverters._ +import scala.util.{Failure, Success, Try, Using} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path @@ -39,6 +40,7 @@ import org.apache.hadoop.hive.metastore.conf.MetastoreConf import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.{AUTO_CREATE_ALL, METASTORE_METADATA_TRANSFORMER_CLASS, SCHEMA_VERIFICATION, STATS_AUTO_GATHER} import org.apache.hadoop.hive.ql.IDriver import org.apache.hadoop.hive.ql.metadata.{Hive, HiveException, Partition => HivePartition, Table => HiveTable} +import org.apache.hadoop.hive.ql.processors.CommandProcessorException import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.ql.util.DirectionUtils import org.apache.hadoop.hive.serde.serdeConstants @@ -889,9 +891,9 @@ private[hive] class HiveClientImpl( val tokens: Array[String] = cmd_trimmed.split("\\s+") // The remainder of the command. val cmd_1: String = cmd_trimmed.substring(tokens(0).length()).trim() - val proc = shim.getCommandProcessor(tokens(0), conf) - proc match { + val results: Try[Seq[String]] = Using(shim.getCommandProcessor(tokens(0), conf)) { case driver: IDriver => + val out = state.getClass.getField("out").get(state) try { driver.run(cmd) driver.setMaxRows(maxRows) @@ -909,7 +911,6 @@ private[hive] class HiveClientImpl( } case _ => - val out = state.getClass.getField("out").get(state) if (out != null) { // scalastyle:off println out.asInstanceOf[PrintStream].println(tokens(0) + " " + cmd_1) @@ -918,6 +919,20 @@ private[hive] class HiveClientImpl( proc.run(cmd_1) Seq.empty } + results match { + case Success(s) => + s + case Failure(e) => + e match { + case e@(_: QueryExecutionException | _: SparkThrowable) => + throw e + case e: CommandProcessorException => + // Wrap the original hive error with QueryExecutionException and throw it + // if there is an error in query processing. + // This works for hive 4.x and later versions. + throw new QueryExecutionException(ExceptionUtils.getStackTrace(e)) + } + } } catch { case e: Exception => logError( @@ -1345,6 +1360,8 @@ private[hive] object HiveClientImpl extends Logging { SQLConf.get.redactOptions(confMap).foreach { case (k, v) => logDebug(s"Applying Hadoop/Hive/Spark and extra properties to Hive Conf:$k=$v") } + // Since HIVE-17626(Hive 3.0.0), need to set hive.query.reexecution.enabled=false. + HiveConf.setBoolVar(hiveConf, HiveConf.ConfVars.HIVE_QUERY_REEXECUTION_ENABLED, false) // Disable CBO because we removed the Calcite dependency. hiveConf.setBoolean("hive.cbo.enable", false) // If this is true, SessionState.start will create a file to log hive job which will not be diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationExec.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationExec.scala index 4535e723c932..0ad77bff8e35 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationExec.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveScriptTransformationExec.scala @@ -145,7 +145,8 @@ private[hive] case class HiveScriptTransformationExec( val (outputStream, proc, inputStream, stderrBuffer) = initProc - val (inputSerde, inputSoi) = initInputSerDe(ioschema, child.output).getOrElse((null, null)) + val (inputSerde, inputSoi) = + initInputSerDe(hadoopConf, ioschema, child.output).getOrElse((null, null)) // For HiveScriptTransformationExec, if inputSerde == null, but outputSerde != null // We will use StringBuffer to pass data, in this case, we should cast data as string too. @@ -173,7 +174,7 @@ private[hive] case class HiveScriptTransformationExec( ) val (outputSerde, outputSoi) = { - initOutputSerDe(ioschema, output).getOrElse((null, null)) + initOutputSerDe(hadoopConf, ioschema, output).getOrElse((null, null)) } val outputIterator = if (outputSerde == null) { @@ -239,11 +240,12 @@ private[hive] case class HiveScriptTransformationWriterThread( object HiveScriptIOSchema extends HiveInspectors { def initInputSerDe( + hadoopConf: Configuration, ioschema: ScriptTransformationIOSchema, input: Seq[Expression]): Option[(AbstractSerDe, StructObjectInspector)] = { ioschema.inputSerdeClass.map { serdeClass => val (columns, columnTypes) = parseAttrs(input) - val serde = initSerDe(serdeClass, columns, columnTypes, ioschema.inputSerdeProps) + val serde = initSerDe(hadoopConf, serdeClass, columns, columnTypes, ioschema.inputSerdeProps) val fieldObjectInspectors = columnTypes.map(toInspector) val objectInspector = ObjectInspectorFactory .getStandardStructObjectInspector(columns.asJava, fieldObjectInspectors.asJava) @@ -252,11 +254,12 @@ object HiveScriptIOSchema extends HiveInspectors { } def initOutputSerDe( + hadoopConf: Configuration, ioschema: ScriptTransformationIOSchema, output: Seq[Attribute]): Option[(AbstractSerDe, StructObjectInspector)] = { ioschema.outputSerdeClass.map { serdeClass => val (columns, columnTypes) = parseAttrs(output) - val serde = initSerDe(serdeClass, columns, columnTypes, ioschema.outputSerdeProps) + val serde = initSerDe(hadoopConf, serdeClass, columns, columnTypes, ioschema.outputSerdeProps) val structObjectInspector = serde.getObjectInspector().asInstanceOf[StructObjectInspector] (serde, structObjectInspector) } @@ -269,6 +272,7 @@ object HiveScriptIOSchema extends HiveInspectors { } def initSerDe( + hadoopConf: Configuration, serdeClassName: String, columns: Seq[String], columnTypes: Seq[DataType], @@ -284,7 +288,7 @@ object HiveScriptIOSchema extends HiveInspectors { val properties = new Properties() properties.putAll(propsMap.asJava) - serde.initialize(null, properties, null) + serde.initialize(hadoopConf, properties, null) serde } diff --git a/sql/hive/src/test/noclasspath/pom.xml b/sql/hive/src/test/noclasspath/pom.xml index 65a7a5f39e55..257e65646ee1 100644 --- a/sql/hive/src/test/noclasspath/pom.xml +++ b/sql/hive/src/test/noclasspath/pom.xml @@ -37,8 +37,9 @@ 8 8 + 8 3.4.1 - 2.3.10 + 4.1.0-SNAPSHOT @@ -46,6 +47,7 @@ maven-jar-plugin + 3.4.2 ${project.basedir} @@ -57,32 +59,16 @@ org.apache.hive hive-exec - ${hive.version} core - provided - - - org.apache.hive - hive-serde ${hive.version} provided org.apache.hive - hive-storage-api - 2.4.0 + hive-udf + ${hive.version} provided - - org.apache.hadoop - hadoop-common - ${hadoop.version} - - - commons-logging - commons-logging - 1.1.3 - diff --git a/sql/hive/src/test/noclasspath/src/main/java/org/apache/hadoop/hive/contrib/udf/example/GenericUDFTrim2.java b/sql/hive/src/test/noclasspath/src/main/java/org/apache/hadoop/hive/contrib/udf/example/GenericUDFTrim2.java index 4a6f05e350ac..0b6432fac3a4 100644 --- a/sql/hive/src/test/noclasspath/src/main/java/org/apache/hadoop/hive/contrib/udf/example/GenericUDFTrim2.java +++ b/sql/hive/src/test/noclasspath/src/main/java/org/apache/hadoop/hive/contrib/udf/example/GenericUDFTrim2.java @@ -21,7 +21,8 @@ import org.apache.commons.lang.StringUtils; import org.apache.hadoop.hive.ql.exec.Description; import org.apache.hadoop.hive.ql.exec.vector.VectorizedExpressions; -import org.apache.hadoop.hive.ql.exec.vector.expressions.StringTrim; +import org.apache.hadoop.hive.ql.exec.vector.expressions.StringTrimCol; +import org.apache.hadoop.hive.ql.exec.vector.expressions.StringTrimColScalar; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBaseTrim; /** @@ -32,15 +33,15 @@ value = "_FUNC_(str) - Removes the leading and trailing space characters from str ", extended = "Example:\n" + " > SELECT _FUNC_(' facebook ') FROM src LIMIT 1;\n" + " 'facebook'") -@VectorizedExpressions({ StringTrim.class }) +@VectorizedExpressions({ StringTrimCol.class, StringTrimColScalar.class }) public class GenericUDFTrim2 extends GenericUDFBaseTrim { public GenericUDFTrim2() { super("generic_udf_trim2"); } @Override - protected String performOp(String val) { - return StringUtils.strip(val, " "); + protected String performOp(String val, String trimChars) { + return StringUtils.strip(val, trimChars); } } \ No newline at end of file diff --git a/sql/hive/src/test/resources/TestUDTF.jar b/sql/hive/src/test/resources/TestUDTF.jar new file mode 100644 index 0000000000000000000000000000000000000000..514f2d5d26fd358ad5647e0e75edb8ce77b69e30 GIT binary patch literal 1328 zcmWIWW@Zs#;Nak3xSr7K#()Gk8CV#6T|*poJ^kGD|D9rBU}gyLX6FE@V1g9+ zP&U6PT|coPF*zeuzqlZ=C|kd{Fh@Tlvn*9VwIVgSv?Mb>Pv1Q?FSRH$In*V@%{jj` zuf#|%IVZ8WcxlK>KjA=;e|ow+J<2CuHJwts5Oyj+z|m2lF^e}kXqra-9v}VG4QH-I z3e+&hyZ7o`e8K!9;8^kPO%YMTk5r1!#h&{#_wB!5f8VM%C>{yDve5geZOQcoVzyjW zs{AWm4sj`IiIjvs-uI-|v|~|xYaioEfnb9b2_6TT+q@2c)9qham^b@gU-(}S?-_u{zB$yU{&~h%g<;U5!%kr`i_rG&( z1=D`Ixp}xt-d0_A?wzF4>zUi0YDp~H8)9C0?2p5AdFR)~+a_PDoF*OZbpMg&?Q^f9 zpRKzP`*6vT1$oc^%$hb^HO-n;rgQS&h#!-Fd7l5syZ%O3Q#+5>wT-+Fck$>h6k2#X zcLjSfwFen_YXhGm z^=`M1UD-x;p7>=k`xz!GhN&ekd9&Q?Yf0SWJ!;Oue2X$)dsxC;)*?8E^F7HIkJ4tmi@{)@8^X5ZB1KR^!8JazLNx?@F#6Qh=oqF?F^3mU?`(NA31m#>^_H^wjyAyleIZG~N zN3cs-gj`tdxtaav#WjMwmn?hLF23F(t;XrL$Fhh;j8kUuBZfwg5A03sEB^S1^O$#B zQ*zkSJH>jz!pVO5lRCTRbb9WZtQMslA1J9HGa0HDESi2z@s87s ze>SVae&s**JNU+?Q1kX**;A{PL(Yr;aEf<6t9WJW3-OqVd-SV#-d isMetastoreSetting(e._1)) // Always add these two metastore settings at the beginning. - (HiveUtils.HIVE_METASTORE_VERSION.key -> "2.3.10") +: + (HiveUtils.HIVE_METASTORE_VERSION.key -> "4.1.0") +: (HiveUtils.HIVE_METASTORE_JARS.key -> "maven") +: filteredSettings } @@ -727,7 +727,7 @@ object SPARK_9757 extends QueryTest { val hiveWarehouseLocation = Utils.createTempDir() val sparkContext = new SparkContext( new SparkConf() - .set(HiveUtils.HIVE_METASTORE_VERSION.key, "2.3.10") + .set(HiveUtils.HIVE_METASTORE_VERSION.key, "4.1.0") .set(HiveUtils.HIVE_METASTORE_JARS.key, "maven") .set(UI_ENABLED, false) .set(WAREHOUSE_PATH.key, hiveWarehouseLocation.toString)) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUtilsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUtilsSuite.scala index ac0618be73fc..1513ba557a35 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUtilsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUtilsSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.hive import java.io.File import java.net.URI -import org.apache.hadoop.hive.conf.HiveConf.ConfVars +import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars import org.apache.spark.{SparkConf, TestUtils} import org.apache.spark.deploy.SparkHadoopUtil @@ -35,9 +35,9 @@ class HiveUtilsSuite extends QueryTest with SQLTestUtils with TestHiveSingleton test("newTemporaryConfiguration overwrites listener configurations") { Seq(true, false).foreach { useInMemoryDerby => val conf = HiveUtils.newTemporaryConfiguration(useInMemoryDerby) - assert(conf(ConfVars.METASTORE_PRE_EVENT_LISTENERS.varname) === "") - assert(conf(ConfVars.METASTORE_EVENT_LISTENERS.varname) === "") - assert(conf(ConfVars.METASTORE_END_FUNCTION_LISTENERS.varname) === "") + assert(conf(ConfVars.EVENT_LISTENERS.getVarname) === "") + assert(conf(ConfVars.EVENT_LISTENERS.getVarname) === "") + assert(conf(ConfVars.END_FUNCTION_LISTENERS.getVarname) === "") } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HadoopVersionInfoSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HadoopVersionInfoSuite.scala index a90fe4493dee..412eb9ab278a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HadoopVersionInfoSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HadoopVersionInfoSuite.scala @@ -45,7 +45,7 @@ class HadoopVersionInfoSuite extends SparkFunSuite { // Download jars for Hive 2.0 val client = IsolatedClientLoader.forVersion( - hiveMetastoreVersion = "2.0", + hiveMetastoreVersion = "4.1.0", // 3.0.x is chosen because that HADOOP-14067 got fixed in 3.1.0 hadoopVersion = "3.0.3", sparkConf = new SparkConf(), @@ -58,7 +58,7 @@ class HadoopVersionInfoSuite extends SparkFunSuite { .filter(!_.getName.startsWith("org.apache.hadoop_hadoop-")) val sparkConf = new SparkConf() - sparkConf.set(HiveUtils.HIVE_METASTORE_VERSION, "2.0") + sparkConf.set(HiveUtils.HIVE_METASTORE_VERSION, "4.1.0") sparkConf.set( HiveUtils.HIVE_METASTORE_JARS, jars.map(_.getCanonicalPath).mkString(File.pathSeparator)) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveVersionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveVersionSuite.scala index 5e2284fb70b5..ee3a677b886e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveVersionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveVersionSuite.scala @@ -35,7 +35,7 @@ private[client] abstract class HiveVersionSuite(version: String) extends SparkFu hadoopConf.set("datanucleus.autoStartMechanismMode", "ignored") hadoopConf.set("hive.metastore.schema.verification", "false") // Since Hive 3.0, HIVE-19310 skipped `ensureDbInit` if `hive.in.test=false`. - if (version == "3.0" || version == "3.1" || version == "4.0") { + if (version == "3.0" || version == "3.1" || version == "4.1") { hadoopConf.set("hive.in.test", "true") } HiveClientBuilder.buildClient(version, hadoopConf) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index e43d1cdb19d4..dabb08f7ec90 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -25,6 +25,7 @@ import java.util.Locale import scala.jdk.CollectionConverters._ import org.apache.hadoop.fs.Path +import org.apache.hadoop.hive.metastore.Warehouse.{DEFAULT_CATALOG_NAME, DEFAULT_DATABASE_NAME} import org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER import org.mockito.ArgumentMatchers.any import org.mockito.Mockito.{spy, times, verify} @@ -1393,7 +1394,8 @@ class HiveDDLSuite case "false" => Map("database" -> "`spark_catalog`.`default`") case _ => Map( "clazz" -> "org.apache.hadoop.hive.ql.metadata.HiveException", - "msg" -> "MetaException(message:Can not drop default database)") + "msg" -> (s"MetaException(message:Can not drop $DEFAULT_DATABASE_NAME database " + + s"in catalog $DEFAULT_CATALOG_NAME)")) } ) } @@ -1756,7 +1758,7 @@ class HiveDDLSuite sql(s"SELECT * FROM ${targetTable.identifier}")) } - test("create table with the same name as an index table") { + ignore("[Hive 4.x: does not support index] create table with the same name as an index table") { val tabName = "tab1" val indexName = tabName + "_index" withTable(tabName) { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala index 343396fa45d0..ae4fd1db6231 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala @@ -738,7 +738,8 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { val df = sql("SELECT CodeGenHiveGenericUDF(v) from HiveGenericUDFTable") val plan = df.queryExecution.executedPlan assert(plan.isInstanceOf[WholeStageCodegenExec]) - checkAnswer(df, Seq(Row("14ab8df5135825bc9f5ff7c30609f02f"))) + checkAnswer(df, + Seq(Row("95e053f9451c5f73215101a92682782327c69bca5f1450419c606b09b26534ab"))) } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala index c1084dd4ee7f..e4cea1c7954f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala @@ -19,6 +19,9 @@ package org.apache.spark.sql.hive.orc import java.io.File +import org.apache.hadoop.mapred.FileInputFormat +import org.scalatestplus.scalacheck.ScalaCheckDrivenPropertyChecks + import org.apache.spark.sql.{AnalysisException, Row} import org.apache.spark.sql.TestingUDT.{IntervalData, IntervalUDT} import org.apache.spark.sql.execution.datasources.orc.OrcSuite @@ -27,7 +30,8 @@ import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.types._ import org.apache.spark.util.Utils -class HiveOrcSourceSuite extends OrcSuite with TestHiveSingleton { +class HiveOrcSourceSuite extends OrcSuite with TestHiveSingleton + with ScalaCheckDrivenPropertyChecks { override val orcImp: String = "hive" @@ -197,8 +201,10 @@ class HiveOrcSourceSuite extends OrcSuite with TestHiveSingleton { } test("SPARK-25993 CREATE EXTERNAL TABLE with subdirectories") { - Seq(true, false).foreach { convertMetastore => - withSQLConf(HiveUtils.CONVERT_METASTORE_ORC.key -> s"$convertMetastore") { + forAll { (convertMetastore: Boolean, inputDirRecursive: Boolean) => + withSQLConf(HiveUtils.CONVERT_METASTORE_ORC.key -> s"$convertMetastore", + // HIVE-19258 and MAPREDUCE-7086: Hive does not include subdirectories by default + FileInputFormat.INPUT_DIR_RECURSIVE -> s"$inputDirRecursive") { withTempDir { dir => withTable("orc_tbl1", "orc_tbl2", "orc_tbl3") { val orcTblStatement1 = @@ -254,7 +260,7 @@ class HiveOrcSourceSuite extends OrcSuite with TestHiveSingleton { |LOCATION '${s"${dir.getCanonicalPath}"}'""".stripMargin sql(topDirStatement) val topDirSqlStatement = s"SELECT * FROM tbl1" - if (convertMetastore) { + if (convertMetastore || !inputDirRecursive) { checkAnswer(sql(topDirSqlStatement), Nil) } else { checkAnswer(sql(topDirSqlStatement), (1 to 6).map(i => Row(i, i, s"orc$i"))) @@ -270,7 +276,7 @@ class HiveOrcSourceSuite extends OrcSuite with TestHiveSingleton { |LOCATION '${s"${dir.getCanonicalPath}/l1/"}'""".stripMargin sql(l1DirStatement) val l1DirSqlStatement = s"SELECT * FROM tbl2" - if (convertMetastore) { + if (convertMetastore || !inputDirRecursive) { checkAnswer(sql(l1DirSqlStatement), (1 to 2).map(i => Row(i, i, s"orc$i"))) } else { checkAnswer(sql(l1DirSqlStatement), (1 to 6).map(i => Row(i, i, s"orc$i"))) @@ -286,7 +292,7 @@ class HiveOrcSourceSuite extends OrcSuite with TestHiveSingleton { |LOCATION '${s"${dir.getCanonicalPath}/l1/l2/"}'""".stripMargin sql(l2DirStatement) val l2DirSqlStatement = s"SELECT * FROM tbl3" - if (convertMetastore) { + if (convertMetastore || !inputDirRecursive) { checkAnswer(sql(l2DirSqlStatement), (3 to 4).map(i => Row(i, i, s"orc$i"))) } else { checkAnswer(sql(l2DirSqlStatement), (3 to 6).map(i => Row(i, i, s"orc$i"))) @@ -305,6 +311,7 @@ class HiveOrcSourceSuite extends OrcSuite with TestHiveSingleton { if (convertMetastore) { checkAnswer(sql(wildcardTopDirSqlStatement), (1 to 2).map(i => Row(i, i, s"orc$i"))) } else { + // * in the path never includes subdirectories checkAnswer(sql(wildcardTopDirSqlStatement), Nil) } @@ -321,6 +328,7 @@ class HiveOrcSourceSuite extends OrcSuite with TestHiveSingleton { if (convertMetastore) { checkAnswer(sql(wildcardL1DirSqlStatement), (1 to 4).map(i => Row(i, i, s"orc$i"))) } else { + // * in the path never includes subdirectories checkAnswer(sql(wildcardL1DirSqlStatement), Nil) } @@ -337,6 +345,7 @@ class HiveOrcSourceSuite extends OrcSuite with TestHiveSingleton { if (convertMetastore) { checkAnswer(sql(wildcardL2SqlStatement), (3 to 6).map(i => Row(i, i, s"orc$i"))) } else { + // * in the path never includes subdirectories checkAnswer(sql(wildcardL2SqlStatement), Nil) } } From 83ef9b700327298cc71ca7e9a1408b76f3ad7ba7 Mon Sep 17 00:00:00 2001 From: Vlad Rozov Date: Sun, 15 Jun 2025 08:54:26 -0700 Subject: [PATCH 08/19] fixes for Java 17 and more --- .../spark/deploy/worker/ExecutorRunner.scala | 15 ++-- .../spark/deploy/SparkSubmitTestUtils.scala | 1 - pom.xml | 4 +- .../sql/catalyst/util/HiveDateTimeUtils.scala | 7 +- .../datasources/DaysWritableV2.scala | 90 +++++++++++++++++++ .../execution/HiveCompatibilitySuite.scala | 4 +- .../spark/sql/hive/HiveInspectors.scala | 5 +- .../spark/sql/hive/client/package.scala | 8 +- .../spark/sql/hive/orc/OrcFileFormat.scala | 1 + .../HiveExternalCatalogVersionsSuite.scala | 2 +- 10 files changed, 113 insertions(+), 24 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DaysWritableV2.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 75d3c7caa660..8f0b684a93e8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -80,16 +80,13 @@ private[deploy] class ExecutorRunner( } workerThread.start() // Shutdown hook that kills actors on shutdown. - if (state == ExecutorState.LAUNCHING || state == ExecutorState.RUNNING) { - shutdownHook = ShutdownHookManager.addShutdownHook { () => - // It's possible that we arrive here before calling `fetchAndRunExecutor`, then `state` will - // be `ExecutorState.LAUNCHING`. In this case, we should set `state` to `FAILED`. - if (state == ExecutorState.LAUNCHING || state == ExecutorState.RUNNING) { - state = ExecutorState.FAILED - } - killProcess("Worker shutting down") + shutdownHook = ShutdownHookManager.addShutdownHook { () => + // It's possible that we arrive here before calling `fetchAndRunExecutor`, then `state` will + // be `ExecutorState.LAUNCHING`. In this case, we should set `state` to `FAILED`. + if (state == ExecutorState.LAUNCHING || state == ExecutorState.RUNNING) { + state = ExecutorState.FAILED } - } + killProcess("Worker shutting down") } } /** diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitTestUtils.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitTestUtils.scala index b9103d74bbb9..e38be4cefc07 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitTestUtils.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitTestUtils.scala @@ -59,7 +59,6 @@ trait SparkSubmitTestUtils extends SparkFunSuite with TimeLimits { val env = builder.environment() if (isSparkTesting) { env.put("SPARK_TESTING", "1") - // env.put("DEFAULT_ARTIFACT_REPOSITORY", "file:///local/home/vrozov/.m2/repository/") } else { env.remove("SPARK_TESTING") env.remove("SPARK_SQL_TESTING") diff --git a/pom.xml b/pom.xml index e5464717b419..447883b195db 100644 --- a/pom.xml +++ b/pom.xml @@ -196,7 +196,7 @@ 3.18.0 2.12.1 - 5.2.10 + 6.0.11 33.4.0-jre 2.11.0 3.1.9 @@ -2544,7 +2544,7 @@ org.datanucleus datanucleus-api-jdo - ${datanucleus-core.version} + 6.0.5 org.apache.thrift diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/util/HiveDateTimeUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/util/HiveDateTimeUtils.scala index f409281da63e..76572ae6fe4b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/util/HiveDateTimeUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/util/HiveDateTimeUtils.scala @@ -28,8 +28,9 @@ object HiveDateTimeUtils { private val zoneId = ZoneId.systemDefault() private def toSqlTimestamp(t: HiveTimestamp): SqlTimestamp = { - val ts = new SqlTimestamp(t.toEpochMilli(zoneId)) - ts.setNanos (t.getNanos) + val millis = t.toEpochMilli(zoneId) + val ts = new SqlTimestamp(millis) + ts.setNanos(t.getNanos) ts } @@ -43,7 +44,7 @@ object HiveDateTimeUtils { def toHiveTimestamp(t: Long): HiveTimestamp = { val javaTimestamp = DateTimeUtils.toJavaTimestamp(t) - val hiveTimestamp = HiveTimestamp.ofEpochMilli(javaTimestamp.getTime, zoneId) + val hiveTimestamp = new HiveTimestamp(javaTimestamp.toLocalDateTime) hiveTimestamp.setNanos(javaTimestamp.getNanos) hiveTimestamp } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DaysWritableV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DaysWritableV2.scala new file mode 100644 index 000000000000..a7c4f78452b2 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DaysWritableV2.scala @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +import java.io.{DataInput, DataOutput, IOException} + +import org.apache.hadoop.hive.common.`type`.Date +import org.apache.hadoop.hive.serde2.io.DateWritableV2 + +import org.apache.spark.sql.catalyst.util.RebaseDateTime.{rebaseGregorianToJulianDays, rebaseJulianToGregorianDays} + +/** + * The class accepts/returns days in Gregorian calendar and rebase them + * via conversion to local date in Julian calendar for dates before 1582-10-15 + * in read/write for backward compatibility with Spark 2.4 and earlier versions. + * + * @param gregorianDays The number of days since the epoch 1970-01-01 in + * Gregorian calendar. + * @param julianDays The number of days since the epoch 1970-01-01 in + * Julian calendar. + */ +class DaysWritableV2 extends DateWritableV2 { + private var gregorianDays: Date = null + + def this(d: Date) = { + this() + gregorianDays = d + super.set(rebaseGregorianToJulianDays(d.toEpochDay)) + } + + def this(d: Int) = { + this(Date.ofEpochDay(d)) + super.set(rebaseGregorianToJulianDays(d)) + } + + def this(dateWritableV2: DateWritableV2) = { + this() + gregorianDays = dateWritableV2 match { + case daysWritableV2: DaysWritableV2 => daysWritableV2.gregorianDays + case dateWritableV2: DateWritableV2 => + Date.ofEpochDay(rebaseJulianToGregorianDays(dateWritableV2.getDays)) + } + super.set(rebaseGregorianToJulianDays(gregorianDays.toEpochDay)) + } + + def getGregorianDays: Int = gregorianDays.toEpochDay + + override def getDays: Int = super.getDays + + override def get: Date = { + super.get() + } + + override def set(d: Date): Unit = { + gregorianDays = d + super.set(rebaseGregorianToJulianDays(d.toEpochDay)) + } + + override def set(d: Int): Unit = { + gregorianDays = Date.ofEpochDay(d) + super.set(rebaseGregorianToJulianDays(d)) + } + + @throws[IOException] + override def write(out: DataOutput): Unit = { + super.write(out) + } + + @throws[IOException] + override def readFields(in: DataInput): Unit = { + super.readFields(in) + gregorianDays = Date.ofEpochDay(rebaseJulianToGregorianDays(getDays)) + } + +} diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 1ccc5cd4efb3..08becc04301c 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -218,7 +218,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "alter1", "input16", - // Hive 4 does not support DynamicSerDe + // HIVE-23483: Hive 4 does not support DynamicSerDe + "implicit_cast1", "input19", // No support for unpublished test udfs. @@ -737,7 +738,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "groupby_ppr", "groupby_sort_6", "having", - "implicit_cast1", "index_serde", "infer_bucket_sort_dyn_part", "innerjoin", diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index 14993ae31824..31712e6a2010 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -35,6 +35,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.errors.DataTypeErrors.toSQLType +import org.apache.spark.sql.execution.datasources.DaysWritableV2 import org.apache.spark.sql.types import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -634,7 +635,7 @@ private[hive] trait HiveInspectors { case x: DateObjectInspector if x.preferWritable() => data: Any => { if (data != null) { - HiveDateTimeUtils.fromHiveDate(x.getPrimitiveWritableObject(data).get()) + new DaysWritableV2(x.getPrimitiveWritableObject(data)).getGregorianDays } else { null } @@ -1099,7 +1100,7 @@ private[hive] trait HiveInspectors { if (value == null) { null } else { - new hiveIo.DateWritableV2(HiveDateTimeUtils.toHiveDate(value.asInstanceOf[Int])) + new DaysWritableV2(value.asInstanceOf[Int]) } private def getTimestampWritableV2(value: Any): hiveIo.TimestampWritableV2 = diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/package.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/package.scala index 1657083153c4..e424f503862a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/package.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/package.scala @@ -102,10 +102,10 @@ package object client { "org.apache.hive:hive-vector-code-gen")) case object v4_1 extends HiveVersion("4.1.0", - extraDeps = Seq("org.apache.hadoop:hadoop-hdfs:3.3.6", - "org.datanucleus:datanucleus-api-jdo:5.2.10", - "org.datanucleus:datanucleus-rdbms:5.2.10", - "org.datanucleus:datanucleus-core:5.2.10", + extraDeps = Seq("org.apache.hadoop:hadoop-hdfs:3.4.1", + "org.datanucleus:datanucleus-api-jdo:6.0.5", + "org.datanucleus:datanucleus-rdbms:6.0.10", + "org.datanucleus:datanucleus-core:6.0.11", "org.datanucleus:javax.jdo:3.2.0-release", "org.springframework:spring-core:5.3.21", "org.springframework:spring-jdbc:5.3.21", diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala index b622ea79da9b..2ac80fdaabfa 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala @@ -288,6 +288,7 @@ private[orc] class OrcOutputWriter( } override def write(row: InternalRow): Unit = { + // getOrCreateInternalWriter() recordWriter.write(NullWritable.get(), serializer.serialize(row)) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala index 6581d39c707e..cf7c74881f37 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala @@ -216,7 +216,7 @@ class HiveExternalCatalogVersionsSuite extends SparkSubmitTestUtils { tryDownloadSpark(version, sparkTestingDir.getCanonicalPath) } - val hiveMetastoreVersion = """^\d+\.\d+""".r.findFirstIn(hiveVersion).get + val hiveMetastoreVersion = "2.3" val args = Seq( "--name", "prepare testing tables", "--master", "local[2]", From a49ef3c0216aa3d955cd685d7b08061509593907 Mon Sep 17 00:00:00 2001 From: Vlad Rozov Date: Tue, 15 Jul 2025 14:17:55 -0700 Subject: [PATCH 09/19] more fixes --- .../sql/catalyst/parser/AstBuilder.scala | 3 +- .../catalog/ExternalCatalogSuite.scala | 8 +++-- .../sql/catalyst/parser/DDLParserSuite.scala | 2 +- .../sql/catalyst/util/HiveDateTimeUtils.scala | 19 ++++++++---- .../command/PlanResolutionSuite.scala | 2 +- .../spark/sql/hive/HiveExternalCatalog.scala | 11 +++---- .../spark/sql/hive/HiveInspectors.scala | 3 +- .../sql/hive/client/HiveClientImpl.scala | 29 ++++++++++++------- .../spark/sql/hive/client/HiveShim.scala | 1 - .../sql/hive/execution/HiveOptions.scala | 3 +- .../spark/sql/hive/orc/OrcFileFormat.scala | 10 +++++-- .../AlterNamespaceSetLocationSuite.scala | 3 +- .../command/ShowCreateTableSuite.scala | 2 +- 13 files changed, 58 insertions(+), 38 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 0ed7bdb1f282..86100c103d6e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -4722,8 +4722,7 @@ class AstBuilder extends DataTypeAstBuilder entry("field.delim", ctx.fieldsTerminatedBy) ++ entry("serialization.format", ctx.fieldsTerminatedBy) ++ entry("escape.delim", ctx.escapedBy) ++ - // The following typo is inherited from Hive... - entry("colelction.delim", ctx.collectionItemsTerminatedBy) ++ + entry("collection.delim", ctx.collectionItemsTerminatedBy) ++ entry("mapkey.delim", ctx.keysTerminatedBy) ++ Option(ctx.linesSeparatedBy).toSeq.map { token => val value = string(visitStringLit(token)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala index 5bb81873449c..60baddc452ea 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala @@ -901,8 +901,12 @@ abstract class ExternalCatalogSuite extends SparkFunSuite { assert(exists(db.locationUri, "my_table")) catalog.renameTable("db1", "my_table", "your_table") - assert(!exists(db.locationUri, "my_table")) - assert(exists(db.locationUri, "your_table")) + // Hive does not rename location for managed tables if the location was specified when table + // was created. + if (this.isInstanceOf[InMemoryCatalogSuite]) { + assert(!exists(db.locationUri, "my_table")) + assert(exists(db.locationUri, "your_table")) + } catalog.dropTable("db1", "your_table", ignoreIfNotExists = false, purge = false) assert(!exists(db.locationUri, "your_table")) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 0cb6f53f273a..78bb75ce773b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -525,7 +525,7 @@ class DDLParserSuite extends AnalysisTest { None, Some(SerdeInfo(storedAs = Some("textfile"), serdeProperties = Map( "field.delim" -> ",", "serialization.format" -> ",", "escape.delim" -> "\\", - "colelction.delim" -> "#", "mapkey.delim" -> "=", "line.delim" -> "\n" + "collection.delim" -> "#", "mapkey.delim" -> "=", "line.delim" -> "\n" )))) Seq(createSql, replaceSql).foreach { sql => testCreateOrReplaceDdl(sql, expectedTableSpec, expectedIfNotExists = false) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/util/HiveDateTimeUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/util/HiveDateTimeUtils.scala index 76572ae6fe4b..a5c23642e352 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/util/HiveDateTimeUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/util/HiveDateTimeUtils.scala @@ -18,11 +18,13 @@ package org.apache.spark.sql.catalyst.util import java.sql.{Timestamp => SqlTimestamp} -import java.time.ZoneId +import java.time.{Instant, LocalDate, LocalDateTime, ZoneId} +import java.time.ZoneOffset.UTC import org.apache.hadoop.hive.common.`type`.{Date => HiveDate, Timestamp => HiveTimestamp} -import org.apache.spark.sql.catalyst.util.RebaseDateTime +import org.apache.spark.sql.catalyst.util.RebaseDateTime.rebaseJulianToGregorianDays +import org.apache.spark.sql.catalyst.util.SparkDateTimeUtils.instantToMicros object HiveDateTimeUtils { private val zoneId = ZoneId.systemDefault() @@ -35,7 +37,12 @@ object HiveDateTimeUtils { } def fromHiveTimestamp(t: HiveTimestamp): Long = { - DateTimeUtils.fromJavaTimestamp(toSqlTimestamp(t)) + // get Hive localDateTime + var localDateTime = LocalDateTime.ofInstant(Instant.ofEpochSecond(t.toEpochSecond, t.getNanos), + UTC) + val julianDate = rebaseJulianToGregorianDays(localDateTime.toLocalDate.toEpochDay.toInt) + localDateTime = LocalDateTime.of(LocalDate.ofEpochDay(julianDate), localDateTime.toLocalTime) + instantToMicros(localDateTime.toInstant(zoneId.getRules.getOffset(localDateTime))) } def fromHiveDate(d: HiveDate): Int = { @@ -45,13 +52,13 @@ object HiveDateTimeUtils { def toHiveTimestamp(t: Long): HiveTimestamp = { val javaTimestamp = DateTimeUtils.toJavaTimestamp(t) val hiveTimestamp = new HiveTimestamp(javaTimestamp.toLocalDateTime) - hiveTimestamp.setNanos(javaTimestamp.getNanos) + // hiveTimestamp.setNanos(javaTimestamp.getNanos) hiveTimestamp } def toHiveDate(d: Int): HiveDate = { - val julian = RebaseDateTime.rebaseGregorianToJulianDays(d) - HiveDate.ofEpochDay(julian) + // val julian = RebaseDateTime.rebaseGregorianToJulianDays(d) + HiveDate.ofEpochDay(d) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index ccf502d79c00..d86cbd3b393c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -3067,7 +3067,7 @@ class PlanResolutionSuite extends SharedSparkSession with AnalysisTest { "escape.delim" -> "y", "serialization.format" -> "x", "line.delim" -> "\n", - "colelction.delim" -> "a", // yes, it's a typo from Hive :) + "collection.delim" -> "a", "mapkey.delim" -> "b")) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index 2568354d8f59..bacc98dc6f15 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -345,19 +345,16 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat // converts the table metadata to Hive compatible format, i.e. set the serde information. def newHiveCompatibleMetastoreTable(serde: HiveSerDe): CatalogTable = { - val location = if (table.tableType == EXTERNAL) { + if (table.tableType == EXTERNAL) { // When we hit this branch, we are saving an external data source table with hive // compatible format, which means the data source is file-based and must have a `path`. require(table.storage.locationUri.isDefined, "External file-based data source table must have a `path` entry in storage properties.") - Some(table.location) - } else { - None } table.copy( storage = table.storage.copy( - locationUri = location, + locationUri = Some(table.location), inputFormat = serde.inputFormat, outputFormat = serde.outputFormat, serde = serde.serde, @@ -552,8 +549,8 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat // properties, to avoid adding a unnecessary path option for Hive serde tables. val hasPathOption = CaseInsensitiveMap(rawTable.storage.properties).contains("path") val storageWithNewPath = if (rawTable.tableType == MANAGED && hasPathOption) { - // If it's a managed table with path option and we are renaming it, then the path option - // becomes inaccurate and we need to update it according to the new table name. + // If it's a managed table with path option, and we are renaming it, then the path option + // becomes inaccurate, and we need to update it according to the new table name. val newTablePath = defaultTablePath(TableIdentifier(newName, Some(db))) updateLocationInStorageProps(rawTable, Some(newTablePath)) } else { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index 31712e6a2010..fc7384e010e3 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -1107,7 +1107,8 @@ private[hive] trait HiveInspectors { if (value == null) { null } else { - new hiveIo.TimestampWritableV2(HiveDateTimeUtils.toHiveTimestamp(value.asInstanceOf[Long])) + new hiveIo.TimestampWritableV2( + HiveDateTimeUtils.toHiveTimestamp(value.asInstanceOf[Long])) } private def getHiveIntervalDayTimeWritable(value: Any): hiveIo.HiveIntervalDayTimeWritable = diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index 206edc2a59e5..29f8046c2435 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -1184,18 +1184,25 @@ private[hive] object HiveClientImpl extends Logging { } table.bucketSpec match { - case Some(bucketSpec) if !HiveExternalCatalog.isDatasourceTable(table) => - hiveTable.setNumBuckets(bucketSpec.numBuckets) - hiveTable.setBucketCols(bucketSpec.bucketColumnNames.toList.asJava) - - if (bucketSpec.sortColumnNames.nonEmpty) { - hiveTable.setSortCols( - bucketSpec.sortColumnNames - .map(col => new Order(col, DirectionUtils.ASCENDING_CODE)) - .toList - .asJava - ) + case Some(bucketSpec) => + def setBuckets(): Unit = { + if (HiveExternalCatalog.isDatasourceTable(table)) { + hiveTable.setNumBuckets(0) + } else { + hiveTable.setNumBuckets(bucketSpec.numBuckets) + hiveTable.setBucketCols(bucketSpec.bucketColumnNames.toList.asJava) + + if (bucketSpec.sortColumnNames.nonEmpty) { + hiveTable.setSortCols( + bucketSpec.sortColumnNames + .map(col => new Order(col, DirectionUtils.ASCENDING_CODE)) + .toList + .asJava + ) + } + } } + setBuckets() case _ => } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala index 5e6a2f11e76c..404b0a1220b7 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala @@ -907,7 +907,6 @@ private[client] class Shim_v2_0 extends Shim with Logging { override def createTable(hive: Hive, table: Table, ifNotExists: Boolean): Unit = { recordHiveCall() - table.setNumBuckets(0) hive.createTable(table, ifNotExists) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveOptions.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveOptions.scala index 0f2eee6798ea..15f10bbd7911 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveOptions.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveOptions.scala @@ -107,8 +107,7 @@ object HiveOptions { val delimiterOptions = Map( "fieldDelim" -> "field.delim", "escapeDelim" -> "escape.delim", - // The following typo is inherited from Hive... - "collectionDelim" -> "colelction.delim", + "collectionDelim" -> "collection.delim", "mapkeyDelim" -> "mapkey.delim", "lineDelim" -> "line.delim").map { case (k, v) => k.toLowerCase(Locale.ROOT) -> v } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala index 2ac80fdaabfa..f5bce5acacbb 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala @@ -170,7 +170,11 @@ case class OrcFileFormat() extends FileFormat with DataSourceRegister with Seria // ObjectInspector during recordReader creation itself and can // avoid NameNode call in unwrapOrcStructs per file. // Specifically would be helpful for partitioned datasets. - val orcReader = OrcFile.createReader(filePath, OrcFile.readerOptions(conf)) + val orcReader = OrcFile.createReader(filePath, + // Hive blindly assumes that Timestamps are in UTC time zone + OrcFile.readerOptions(conf) + .convertToProlepticGregorian(false) + .useUTCTimestamp(true)) new SparkOrcNewRecordReader(orcReader, conf, file.start, file.length) } @@ -309,7 +313,9 @@ private[orc] class OrcOutputWriter( // Hive ORC initializes its private `writer` field at the first write. // For empty write task, we need to create it manually to record our meta. val options = OrcFile.writerOptions(context.getConfiguration) - options.inspector(serializer.structOI) + .inspector(serializer.structOI) + // .setProlepticGregorian(true) + // .useUTCTimestamp(true) writer = OrcFile.createWriter(new Path(path), options) // set the writer to make it flush meta on close writerField.set(recordWriter, writer) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterNamespaceSetLocationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterNamespaceSetLocationSuite.scala index 232916b6e05b..126c9dd37fb0 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterNamespaceSetLocationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterNamespaceSetLocationSuite.scala @@ -28,7 +28,8 @@ class AlterNamespaceSetLocationSuite extends v1.AlterNamespaceSetLocationSuiteBa with CommandSuiteBase { override def commandVersion: String = super[AlterNamespaceSetLocationSuiteBase].commandVersion - test("Hive catalog not supported") { + // Hive catalog is supported + ignore("Hive catalog not supported") { val ns = s"$catalog.$namespace" withNamespace(ns) { sql(s"CREATE NAMESPACE $ns") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowCreateTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowCreateTableSuite.scala index 8e654d28cd03..9aed44129b2d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowCreateTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowCreateTableSuite.scala @@ -131,7 +131,7 @@ class ShowCreateTableSuite extends v1.ShowCreateTableSuiteBase with CommandSuite val expected = s"CREATE TABLE $nsTable ( c1 INT COMMENT 'bla', c2 STRING)" + " ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'" + " WITH SERDEPROPERTIES (" + - " 'colelction.delim' = '@'," + + " 'collection.delim' = '@'," + " 'field.delim' = ','," + " 'mapkey.delim' = '#'," + " 'serialization.format' = ',')" + From d7794abbbb1c3570b602ca935e31e17f9872d085 Mon Sep 17 00:00:00 2001 From: Vlad Rozov Date: Mon, 28 Jul 2025 09:43:25 -0700 Subject: [PATCH 10/19] more fixes --- .../sql/execution/datasources/DaysWritableV2.scala | 7 +++---- .../execution/datasources/orc/OrcSourceSuite.scala | 13 ++++++++++--- .../org/apache/spark/sql/hive/HiveInspectors.scala | 8 +++++--- 3 files changed, 18 insertions(+), 10 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DaysWritableV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DaysWritableV2.scala index a7c4f78452b2..c7e3e8f0147e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DaysWritableV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DaysWritableV2.scala @@ -40,12 +40,11 @@ class DaysWritableV2 extends DateWritableV2 { def this(d: Date) = { this() gregorianDays = d - super.set(rebaseGregorianToJulianDays(d.toEpochDay)) + super.set(d) } def this(d: Int) = { this(Date.ofEpochDay(d)) - super.set(rebaseGregorianToJulianDays(d)) } def this(dateWritableV2: DateWritableV2) = { @@ -68,12 +67,12 @@ class DaysWritableV2 extends DateWritableV2 { override def set(d: Date): Unit = { gregorianDays = d - super.set(rebaseGregorianToJulianDays(d.toEpochDay)) + super.set(d.toEpochDay) } override def set(d: Int): Unit = { gregorianDays = Date.ofEpochDay(d) - super.set(rebaseGregorianToJulianDays(d)) + super.set(d) } @throws[IOException] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala index 040999476ece..469cecbdc9e8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala @@ -32,13 +32,13 @@ import org.apache.orc.OrcProto.ColumnEncoding.Kind.{DICTIONARY_V2, DIRECT, DIREC import org.apache.orc.OrcProto.Stream.Kind import org.apache.orc.impl.RecordReaderImpl import org.scalatest.BeforeAndAfterAll - import org.apache.spark.{SPARK_VERSION_SHORT, SparkConf, SparkException} + import org.apache.spark.sql.{Row, SPARK_VERSION_METADATA_KEY} import org.apache.spark.sql.execution.datasources.{CommonFileDataSourceSuite, SchemaMergeUtils} import org.apache.spark.sql.execution.datasources.orc.OrcCompressionCodec._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtilsBase} +import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtilsBase, TestSparkSession} import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -508,7 +508,14 @@ abstract class OrcSuite withAllNativeOrcReaders { checkAnswer( spark.read.orc(path), - Seq(Row(Date.valueOf("1001-01-01")), Row(Date.valueOf("1582-10-15")))) + Seq(Row(Date.valueOf("1001-01-01")), + if (spark.isInstanceOf[TestSparkSession]) { + // Spark rebases 1582-10-05 through 1582-10-15 to 1582-10-15 + Row(Date.valueOf("1582-10-15")) + } else { + // Hive rebases 1582-10-05 through 1582-10-15 by adding 10 days + Row(Date.valueOf("1582-10-20")) + })) } } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index fc7384e010e3..4f4e2df65eb6 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -34,8 +34,8 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.catalyst.util.RebaseDateTime.rebaseJulianToGregorianDays import org.apache.spark.sql.errors.DataTypeErrors.toSQLType -import org.apache.spark.sql.execution.datasources.DaysWritableV2 import org.apache.spark.sql.types import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -635,7 +635,8 @@ private[hive] trait HiveInspectors { case x: DateObjectInspector if x.preferWritable() => data: Any => { if (data != null) { - new DaysWritableV2(x.getPrimitiveWritableObject(data)).getGregorianDays + val d = rebaseJulianToGregorianDays(x.getPrimitiveWritableObject(data).getDays) + d } else { null } @@ -1100,7 +1101,8 @@ private[hive] trait HiveInspectors { if (value == null) { null } else { - new DaysWritableV2(value.asInstanceOf[Int]) + val d = new hiveIo.DateWritableV2(value.asInstanceOf[Int]) + d } private def getTimestampWritableV2(value: Any): hiveIo.TimestampWritableV2 = From 0cccefe0d5db698ca22c110e9e8bfc4522fefb82 Mon Sep 17 00:00:00 2001 From: Vlad Rozov Date: Thu, 31 Jul 2025 16:04:01 -0700 Subject: [PATCH 11/19] Hive 4.1.0 --- pom.xml | 6 +++++- project/SparkBuild.scala | 5 ++++- .../sql/execution/datasources/orc/OrcSourceSuite.scala | 2 +- .../scala/org/apache/spark/sql/hive/client/HiveShim.scala | 2 +- sql/hive/src/test/noclasspath/pom.xml | 2 +- 5 files changed, 12 insertions(+), 5 deletions(-) diff --git a/pom.xml b/pom.xml index 447883b195db..9f0be5e2404f 100644 --- a/pom.xml +++ b/pom.xml @@ -134,7 +134,7 @@ org.apache.hive core - 4.1.0-SNAPSHOT + 4.1.0 3.9.1 @@ -1757,6 +1757,10 @@ ${hive.version} ${hive.deps.scope} + + ${hive.group} + hive-standalone-metastore-common + ${hive.group} hive-shims diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 9be58ea49dbe..304010ce2bd9 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -1091,11 +1091,14 @@ object KubernetesIntegrationTests { object DependencyOverrides { lazy val guavaVersion = sys.props.get("guava.version").getOrElse("33.4.0-jre") lazy val log4jVersion = sys.props.get("log4j.version").getOrElse("2.24.3") + lazy val derbyVersion = sys.props.get("derby.version").getOrElse("10.16.1.1") lazy val settings = Seq( dependencyOverrides += "com.google.guava" % "guava" % guavaVersion, dependencyOverrides += "jline" % "jline" % "2.14.6", dependencyOverrides += "org.apache.avro" % "avro" % "1.12.0", - dependencyOverrides += "org.apache.logging.log4j" % "log4j-slf4j-impl" % log4jVersion) + dependencyOverrides += "org.apache.logging.log4j" % "log4j-slf4j-impl" % log4jVersion, + dependencyOverrides += "org.apache.derby" % "derby" % derbyVersion + ) } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala index 469cecbdc9e8..e9e725956613 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala @@ -32,8 +32,8 @@ import org.apache.orc.OrcProto.ColumnEncoding.Kind.{DICTIONARY_V2, DIRECT, DIREC import org.apache.orc.OrcProto.Stream.Kind import org.apache.orc.impl.RecordReaderImpl import org.scalatest.BeforeAndAfterAll -import org.apache.spark.{SPARK_VERSION_SHORT, SparkConf, SparkException} +import org.apache.spark.{SPARK_VERSION_SHORT, SparkConf, SparkException} import org.apache.spark.sql.{Row, SPARK_VERSION_METADATA_KEY} import org.apache.spark.sql.execution.datasources.{CommonFileDataSourceSuite, SchemaMergeUtils} import org.apache.spark.sql.execution.datasources.orc.OrcCompressionCodec._ diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala index 404b0a1220b7..4ff1e9855fa5 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala @@ -1264,7 +1264,7 @@ private[client] class Shim_v3_0 extends Shim_v2_3 { private[client] class Shim_v3_1 extends Shim_v3_0 -private[client] class Shim_v4_1 extends Shim_v3_1 { +private[client] class Shim_v4_0 extends Shim_v3_1 { private lazy val clazzLoadFileType = getClass.getClassLoader.loadClass( "org.apache.hadoop.hive.ql.plan.LoadTableDesc$LoadFileType") private lazy val clazzLoadTableDesc = getClass.getClassLoader.loadClass( diff --git a/sql/hive/src/test/noclasspath/pom.xml b/sql/hive/src/test/noclasspath/pom.xml index 257e65646ee1..838676e287ca 100644 --- a/sql/hive/src/test/noclasspath/pom.xml +++ b/sql/hive/src/test/noclasspath/pom.xml @@ -39,7 +39,7 @@ 8 8 3.4.1 - 4.1.0-SNAPSHOT + 4.1.0 From 48bc4975c5470116c93d936264934a344e42f72f Mon Sep 17 00:00:00 2001 From: Vlad Rozov Date: Thu, 31 Jul 2025 20:18:37 -0700 Subject: [PATCH 12/19] Revert MavenUtils --- .../main/scala/org/apache/spark/util/MavenUtils.scala | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/common/utils/src/main/scala/org/apache/spark/util/MavenUtils.scala b/common/utils/src/main/scala/org/apache/spark/util/MavenUtils.scala index 82280adec594..5e923ad35a3a 100644 --- a/common/utils/src/main/scala/org/apache/spark/util/MavenUtils.scala +++ b/common/utils/src/main/scala/org/apache/spark/util/MavenUtils.scala @@ -152,17 +152,6 @@ private[spark] object MavenUtils extends Logging { cr.add(localM2) } - if (SparkEnvUtils.isTesting) { - val localM2 = new IBiblioResolver - localM2.setM2compatible(true) - localM2.setRoot( - new File(System.getProperty("user.home"), ".m2" + File.separator + "repository") - .toURI.toString) - localM2.setUsepoms(true) - localM2.setName("user-home-m2-cache") - cr.add(localM2) - } - val localIvy = new FileSystemResolver val localIvyRoot = new File(defaultIvyUserDir, "local") localIvy.setLocal(true) From 49679e6429088441049e305e330182a1b7666941 Mon Sep 17 00:00:00 2001 From: Vlad Rozov Date: Sat, 2 Aug 2025 23:00:00 -0700 Subject: [PATCH 13/19] calcite dependency fix --- pom.xml | 5 +++-- sql/hive/pom.xml | 1 + 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 9f0be5e2404f..0652e62a7439 100644 --- a/pom.xml +++ b/pom.xml @@ -276,6 +276,7 @@ needing Hadoop classes in the classpath to compile). --> compile + compile compile compile ${hive.version} @@ -1878,8 +1879,8 @@ ant - com.esotericsoftware.kryo - kryo + org.datanucleus + datanucleus-core commons-codec diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 72024f49a171..b17a0a6c0b55 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -159,6 +159,7 @@ org.apache.calcite calcite-core + ${calcite.deps.scope} org.apache.hadoop From 6b47aad68239807acdec9b003ec249b690842628 Mon Sep 17 00:00:00 2001 From: Vlad Rozov Date: Sun, 3 Aug 2025 20:26:47 -0700 Subject: [PATCH 14/19] dependencies --- assembly/pom.xml | 1 + pom.xml | 2 +- sql/hive/pom.xml | 3 ++- 3 files changed, 4 insertions(+), 2 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 350168d57f94..34fa9f8077c4 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -339,6 +339,7 @@ provided provided + provided diff --git a/pom.xml b/pom.xml index 0652e62a7439..246cdbc29a2b 100644 --- a/pom.xml +++ b/pom.xml @@ -135,6 +135,7 @@ core 4.1.0 + ${hive.version} 3.9.1 @@ -279,7 +280,6 @@ compile compile compile - ${hive.version} compile compile test diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index b17a0a6c0b55..0ec0891a8fc7 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -155,6 +155,7 @@ ${hive.group} hive-parser + ${hive.deps.scope} org.apache.calcite @@ -165,7 +166,7 @@ org.apache.hadoop hadoop-hdfs ${hadoop.version} - runtime + ${hadoop.deps.scope} From d8390c70b82d87fd893ce64fcdfd25c5f3d09444 Mon Sep 17 00:00:00 2001 From: Vlad Rozov Date: Mon, 4 Aug 2025 11:17:27 -0700 Subject: [PATCH 15/19] dependencies and unused imports --- pom.xml | 4 ++++ .../main/java/org/apache/hive/service/auth/HttpAuthUtils.java | 1 - .../java/org/apache/hive/service/auth/KerberosSaslHelper.java | 1 - .../hive/service/cli/operation/ExecuteStatementOperation.java | 1 - .../apache/hive/service/cli/operation/LogDivertAppender.java | 1 - .../java/org/apache/hive/service/cli/operation/Operation.java | 2 -- 6 files changed, 4 insertions(+), 6 deletions(-) diff --git a/pom.xml b/pom.xml index 246cdbc29a2b..0d77dea0c52e 100644 --- a/pom.xml +++ b/pom.xml @@ -2118,6 +2118,10 @@ com.jolbox bonecp + + org.apache.orc + orc-core + diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java index 489608de72f3..57931e600fcb 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java @@ -32,7 +32,6 @@ import org.apache.commons.codec.binary.Base64; import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge; -import org.apache.hadoop.hive.shims.ShimLoader; import org.apache.hadoop.security.UserGroupInformation; import org.apache.http.protocol.BasicHttpContext; import org.apache.http.protocol.HttpContext; diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java index 4b05d72530dd..e6d425753482 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java @@ -20,7 +20,6 @@ import java.util.Map; import javax.security.sasl.SaslException; -import org.apache.hadoop.hive.shims.ShimLoader; import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge; import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge.Server; import org.apache.hive.service.cli.thrift.ThriftCLIService; diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java index 4c4a3f60e5e7..8de82d0a05e7 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java @@ -18,7 +18,6 @@ import java.util.Map; -import org.apache.hadoop.hive.ql.session.OperationLog; import org.apache.hive.service.cli.OperationType; import org.apache.hive.service.cli.session.HiveSession; diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/LogDivertAppender.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/LogDivertAppender.java index 28023bea477b..d84653b05cd8 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/LogDivertAppender.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/LogDivertAppender.java @@ -16,7 +16,6 @@ */ package org.apache.hive.service.cli.operation; -import java.io.BufferedOutputStream; import java.io.CharArrayWriter; import java.io.FileNotFoundException; import java.io.FileOutputStream; diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/Operation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/Operation.java index 75ad4cd74211..a7675865cdff 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/Operation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/Operation.java @@ -17,7 +17,6 @@ package org.apache.hive.service.cli.operation; import java.io.File; -import java.io.FileNotFoundException; import java.util.EnumSet; import java.util.HashMap; import java.util.Map; @@ -28,7 +27,6 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.QueryState; import org.apache.hadoop.hive.ql.processors.CommandProcessorException; -import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse; import org.apache.hadoop.hive.ql.session.OperationLog; import org.apache.hive.service.cli.FetchOrientation; import org.apache.hive.service.cli.HiveSQLException; From e07303c21e1c864d3f8b3e63b718135b5ff380c4 Mon Sep 17 00:00:00 2001 From: Vlad Rozov Date: Mon, 4 Aug 2025 15:39:55 -0700 Subject: [PATCH 16/19] restrict to 4.1 for now --- .../org/apache/spark/sql/hive/client/HiveClientVersions.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientVersions.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientVersions.scala index c24a065bcb3c..a9fc295911b1 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientVersions.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientVersions.scala @@ -22,6 +22,6 @@ private[client] trait HiveClientVersions { protected val versions = if (testVersions.nonEmpty) { testVersions.get.split(",").map(_.trim).filter(_.nonEmpty).toIndexedSeq } else { - IndexedSeq("4.0", "4.1") + IndexedSeq("4.1") } } From a13a7c0876e911c7539e89b3a2ba694325d271cf Mon Sep 17 00:00:00 2001 From: Vlad Rozov Date: Thu, 7 Aug 2025 21:35:51 -0700 Subject: [PATCH 17/19] exclude orc-core and orc-format from SBT build --- project/SparkBuild.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 304010ce2bd9..dbee5b0c9de6 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -290,6 +290,12 @@ object SparkBuild extends PomBuild { sparkGenjavadocSettings ++ compilerWarningSettings ++ (if (noLintOnCompile) Nil else enableScalaStyle) ++ Seq( + (Compile / dependencyClasspath) := (Compile / dependencyClasspath).value + .filterNot(file => { + val name = file.toString + (name.contains("orc-core") || name.contains("orc-format")) && + !name.contains("shaded-protobuf") + }), (Compile / exportJars) := true, (Test / exportJars) := false, javaHome := sys.env.get("JAVA_HOME") From 76561962e27353c8cb903f66b9b7dc04be0c5274 Mon Sep 17 00:00:00 2001 From: Vlad Rozov Date: Thu, 7 Aug 2025 22:37:01 -0700 Subject: [PATCH 18/19] fix --- .../sql/hive/client/HiveClientImpl.scala | 23 ++++--------------- 1 file changed, 5 insertions(+), 18 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index 29f8046c2435..f2df6a12651d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -893,24 +893,11 @@ private[hive] class HiveClientImpl( val cmd_1: String = cmd_trimmed.substring(tokens(0).length()).trim() val results: Try[Seq[String]] = Using(shim.getCommandProcessor(tokens(0), conf)) { case driver: IDriver => + driver.run(cmd) + driver.setMaxRows(maxRows) + shim.getDriverResults(driver) + case proc => val out = state.getClass.getField("out").get(state) - try { - driver.run(cmd) - driver.setMaxRows(maxRows) - shim.getDriverResults(driver) - } catch { - case e @ (_: QueryExecutionException | _: SparkThrowable) => - throw e - case e: Exception => - // Wrap the original hive error with QueryExecutionException and throw it - // if there is an error in query processing. - // This works for hive 4.x and later versions. - throw new QueryExecutionException(Utils.stackTraceToString(e)) - } finally { - driver.close - } - - case _ => if (out != null) { // scalastyle:off println out.asInstanceOf[PrintStream].println(tokens(0) + " " + cmd_1) @@ -930,7 +917,7 @@ private[hive] class HiveClientImpl( // Wrap the original hive error with QueryExecutionException and throw it // if there is an error in query processing. // This works for hive 4.x and later versions. - throw new QueryExecutionException(ExceptionUtils.getStackTrace(e)) + throw new QueryExecutionException(Utils.stackTraceToString(e)) } } } catch { From 3d17d5b48aae9eaa2a1f5baf4b46d9a704fe4741 Mon Sep 17 00:00:00 2001 From: Vlad Rozov Date: Thu, 7 Aug 2025 22:49:14 -0700 Subject: [PATCH 19/19] fix --- .../org/apache/hive/service/cli/operation/Operation.java | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/Operation.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/Operation.java index a7675865cdff..4d3d9ff337e5 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/Operation.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/Operation.java @@ -240,14 +240,7 @@ protected void createOperationLog() { } // create OperationLog object with above log file - try { - operationLog = new OperationLog(opHandle.toString(), operationLogFile, parentSession.getHiveConf()); - } catch (FileNotFoundException e) { - LOG.warn("Unable to instantiate OperationLog object for operation: {}", e, - MDC.of(LogKeys.OPERATION_HANDLE, opHandle)); - isOperationLogEnabled = false; - return; - } + operationLog = new OperationLog(opHandle.toString(), operationLogFile, parentSession.getHiveConf()); // register this operationLog to current thread operationManager.setCurrentOperationLog(operationLog, operationLogFile);