[jira] [Created] (HIVE-22169) Tez: SplitGenerator tries to look for plan files which won't exist for Tez
Gopal V created HIVE-22169: -- Summary: Tez: SplitGenerator tries to look for plan files which won't exist for Tez Key: HIVE-22169 URL: https://issues.apache.org/jira/browse/HIVE-22169 Project: Hive Issue Type: Bug Reporter: Gopal V {code} at org.apache.hadoop.hive.ql.exec.Utilities.clearWork(Utilities.java:310) at org.apache.hadoop.hive.ql.exec.tez.HiveSplitGenerator.initialize(HiveSplitGenerator.java:318) at org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable$1.run(RootInputInitializerManager.java:278) at org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable$1.run(RootInputInitializerManager.java:269) at java.security.AccessController.doPrivileged(Native Method {code} The split generator tries to clear out the work items from HDFS, which will never exist for Tez plans. -- This message was sent by Atlassian Jira (v8.3.2#803003)
[jira] [Created] (HIVE-22163) CBO: Enabling CBO turns on stats estimation, even when the estimation is disabled
Gopal V created HIVE-22163: -- Summary: CBO: Enabling CBO turns on stats estimation, even when the estimation is disabled Key: HIVE-22163 URL: https://issues.apache.org/jira/browse/HIVE-22163 Project: Hive Issue Type: Bug Reporter: Gopal V {code} create table claims(claim_rec_id bigint, claim_invoice_num string, typ_c int); alter table claims update statistics set ('numRows'='1154941534','rawDataSize'='1135307527922'); set hive.stats.estimate=false; explain extended select count(1) from claims where typ_c=3; set hive.stats.ndv.estimate.percent=5e-7; explain extended select count(1) from claims where typ_c=3; {code} Expecting the standard /2 for the single filter, but we instead get 5 rows. {code} 'Map Operator Tree:' 'TableScan' ' alias: claims' ' filterExpr: (typ_c = 3) (type: boolean)' ' Statistics: Num rows: 1154941534 Data size: 4388777832 Basic stats: COMPLETE Column stats: NONE' ' GatherStats: false' ' Filter Operator' 'isSamplingPred: false' 'predicate: (typ_c = 3) (type: boolean)' 'Statistics: Num rows: 5 Data size: 19 Basic stats: COMPLETE Column stats: NONE' {code} The estimation is in effect, as changing the estimate.percent changes this. {code} ' filterExpr: (typ_c = 3) (type: boolean)' ' Statistics: Num rows: 1154941534 Data size: 4388777832 Basic stats: COMPLETE Column stats: NONE' ' GatherStats: false' ' Filter Operator' 'isSamplingPred: false' 'predicate: (typ_c = 3) (type: boolean)' 'Statistics: Num rows: 230988307 Data size: 877755567 Basic stats: COMPLETE Column stats: NONE' {code} -- This message was sent by Atlassian Jira (v8.3.2#803003)
[jira] [Created] (HIVE-22161) UDF: FunctionRegistry should cache statefulness of UDF
Gopal V created HIVE-22161: -- Summary: UDF: FunctionRegistry should cache statefulness of UDF Key: HIVE-22161 URL: https://issues.apache.org/jira/browse/HIVE-22161 Project: Hive Issue Type: Bug Components: UDF Reporter: Gopal V There's a hidden synchronization across threads when looking up isStateful and isDeterministic. https://github.com/apache/hive/blob/master/common/src/java/org/apache/hive/common/util/AnnotationUtils.java#L27 {code} // to avoid https://bugs.openjdk.java.net/browse/JDK-7122142 public static T getAnnotation(Class clazz, Class annotationClass) { synchronized (annotationClass) { return clazz.getAnnotation(annotationClass); } } {code} This is serializing multiple threads initializing UDFs (or checking them during compilation) & also being locked across threads for each instance of GenericUDFOpEqual in the specific scenario. -- This message was sent by Atlassian Jira (v8.3.2#803003)
[jira] [Created] (HIVE-22106) PCR: Remove cross-query synchronization for the partition-eval
Gopal V created HIVE-22106: -- Summary: PCR: Remove cross-query synchronization for the partition-eval Key: HIVE-22106 URL: https://issues.apache.org/jira/browse/HIVE-22106 Project: Hive Issue Type: Bug Reporter: Gopal V {code} HiveServer2-Handler-Pool: Thread-492 Blocked CPU usage on sample: 0ms org.apache.hadoop.hive.ql.optimizer.ppr.PartExprEvalUtils.evalExprWithPart(ExprNodeDesc, Partition, List, StructObjectInspector) PartExprEvalUtils.java:58 org.apache.hadoop.hive.ql.optimizer.pcr.PcrExprProcFactory.evalExprWithPart(ExprNodeDesc, Partition, List) PcrExprProcFactory.java:83 org.apache.hadoop.hive.ql.optimizer.pcr.PcrExprProcFactory$GenericFuncExprProcessor.handleDeterministicUdf(PcrExprProcCtx, ExprNodeGenericFuncDesc, Object[]) PcrExprProcFactory.java:317 org.apache.hadoop.hive.ql.optimizer.pcr.PcrExprProcFactory$GenericFuncExprProcessor.process(Node, Stack, NodeProcessorCtx, Object[]) PcrExprProcFactory.java:298 org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher.dispatch(Node, Stack, Object[]) DefaultRuleDispatcher.java:90 org.apache.hadoop.hive.ql.lib.DefaultGraphWalker.dispatchAndReturn(Node, Stack) DefaultGraphWalker.java:105 org.apache.hadoop.hive.ql.lib.DefaultGraphWalker.dispatch(Node, Stack) DefaultGraphWalker.java:89 org.apache.hadoop.hive.ql.lib.DefaultGraphWalker.walk(Node) DefaultGraphWalker.java:158 org.apache.hadoop.hive.ql.lib.DefaultGraphWalker.startWalking(Collection, HashMap) DefaultGraphWalker.java:120 {code} -- This message was sent by Atlassian JIRA (v7.6.14#76016)
[jira] [Created] (HIVE-22076) JDK11: Remove ParallelGC in debug.sh
Gopal V created HIVE-22076: -- Summary: JDK11: Remove ParallelGC in debug.sh Key: HIVE-22076 URL: https://issues.apache.org/jira/browse/HIVE-22076 Project: Hive Issue Type: Bug Components: Diagnosability Affects Versions: 4.0.0 Reporter: Gopal V The JDK debug mode no longer depends on ParallelGC -- This message was sent by Atlassian JIRA (v7.6.14#76016)
[jira] [Created] (HIVE-22064) CBO: Rewrite year() predicate to a constant condition
Gopal V created HIVE-22064: -- Summary: CBO: Rewrite year() predicate to a constant condition Key: HIVE-22064 URL: https://issues.apache.org/jira/browse/HIVE-22064 Project: Hive Issue Type: Improvement Reporter: Gopal V {code} CREATE TEMPORARY TABLE users (signup_date date, user_id bigint) stored as ORC; INSERT INTO users values("1999-01-01", 1); EXPLAIN ANALYZE SELECT year(signup_date), count(distinct user_id) from users where year(signup_date) BETWEEN 2017 and 2019 GROUP BY year(signup_date); {code} The YEAR() or EXTRACT( YEAR) is not rewritten into a constant for push-down. {code} EXPLAIN ANALYZE SELECT year(signup_date), count(distinct user_id) from users where signup_date BETWEEN DATE'2017-01-01' and DATE'2019-12-31' GROUP BY year(signup_date); {code} Does push-down into the storage layers. -- This message was sent by Atlassian JIRA (v7.6.14#76016)
[jira] [Created] (HIVE-21985) LLAP IO: Log schema evolution incompatibilities at WARN level always
Gopal V created HIVE-21985: -- Summary: LLAP IO: Log schema evolution incompatibilities at WARN level always Key: HIVE-21985 URL: https://issues.apache.org/jira/browse/HIVE-21985 Project: Hive Issue Type: Bug Reporter: Gopal V When reading incompatible schema, LLAP IO simply skips over the file and does not cache it. The logging at WARN level would be useful and simplify the root-cause via logs. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-21828) Tez: Use a TezStaticConfiguration inherited from TezRuntimeConfiguration
Gopal V created HIVE-21828: -- Summary: Tez: Use a TezStaticConfiguration inherited from TezRuntimeConfiguration Key: HIVE-21828 URL: https://issues.apache.org/jira/browse/HIVE-21828 Project: Hive Issue Type: Bug Reporter: Gopal V The HS2 tez-site.xml does not change dynamically - the XML parsed components of the config can be obtained statically and kept across sessions. This allows for the replacing of "new TezConfiguration()" with a HS2 local version instead. The configuration object however has to reference the right resource file (i.e location of tez-site.xml) without reparsing it for each query. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-21819) HiveConf: differentiate the resource between file configs and explicit sets
Gopal V created HIVE-21819: -- Summary: HiveConf: differentiate the resource between file configs and explicit sets Key: HIVE-21819 URL: https://issues.apache.org/jira/browse/HIVE-21819 Project: Hive Issue Type: Bug Components: Configuration Affects Versions: 4.0.0 Reporter: Gopal V -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-21818) CBO: Copying TableRelOptHiveTable has metastore traffic
Gopal V created HIVE-21818: -- Summary: CBO: Copying TableRelOptHiveTable has metastore traffic Key: HIVE-21818 URL: https://issues.apache.org/jira/browse/HIVE-21818 Project: Hive Issue Type: Bug Reporter: Gopal V While the optimizer is running, whenever it makes a copy of the TableScan to perform potential rewrites, there is Hive metastore traffic from inside CBO optimizer. {code} public RelOptHiveTable(RelOptSchema calciteSchema, RelDataTypeFactory typeFactory, List qualifiedTblName, RelDataType rowType, Table hiveTblMetadata, List hiveNonPartitionCols, List hivePartitionCols, List hiveVirtualCols, HiveConf hconf, Map partitionCache, Map colStatsCache, AtomicInteger noColsMissingStats) { Pair, List> constraintKeys = generateKeys(); this.keys = constraintKeys.left; this.nonNullablekeys = constraintKeys.right; this.referentialConstraints = generateReferentialConstraints(); } {code} This is triggered from the rules as the partition pruner {code} protected void perform(RelOptRuleCall call, Filter filter, HiveTableScan tScan) { // Original table RelOptHiveTable hiveTable = (RelOptHiveTable) tScan.getTable(); // Copy original table scan and table HiveTableScan tScanCopy = tScan.copyIncludingTable(tScan.getRowType()); RelOptHiveTable hiveTableCopy = (RelOptHiveTable) tScanCopy.getTable(); {code} -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-21817) CBO: PartitionPruner cache-miss from Calcite planner
Gopal V created HIVE-21817: -- Summary: CBO: PartitionPruner cache-miss from Calcite planner Key: HIVE-21817 URL: https://issues.apache.org/jira/browse/HIVE-21817 Project: Hive Issue Type: Bug Components: CBO Reporter: Gopal V Attachments: calcite-ppr.png !calcite-ppr.png! {code} public static PrunedPartitionList prune(Table tab, ExprNodeDesc prunerExpr, HiveConf conf, String alias, Map prunedPartitionsMap) throws SemanticException { vs public PrunedPartitionList getPrunedPartitions(String alias, TableScanOperator ts) throws SemanticException { PrunedPartitionList partsList = opToPartList.get(ts); {code} Only the 2nd one hits the PPR cache. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-21805) HiveServer2: Use the fast ShutdownHookManager APIs
Gopal V created HIVE-21805: -- Summary: HiveServer2: Use the fast ShutdownHookManager APIs Key: HIVE-21805 URL: https://issues.apache.org/jira/browse/HIVE-21805 Project: Hive Issue Type: Bug Reporter: Gopal V Attachments: shutdownhookmanager-configuration.png Hadoop ShutDownHookManager calls "new Configuration()" inside if the parameters are not provided as args. This unzips jars & looks for the .xml files in the entire classpath. !shutdownhookmanager-configuration.png! Hive has its own impl of ShutDownHookManager, which has also history from the hadoo one (added to code instead of shims). -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-21800) Tez: Cartesian product reparses HiveConf XML
Gopal V created HIVE-21800: -- Summary: Tez: Cartesian product reparses HiveConf XML Key: HIVE-21800 URL: https://issues.apache.org/jira/browse/HIVE-21800 Project: Hive Issue Type: Bug Components: Tez Reporter: Gopal V Attachments: Tez-CartesianProductSlowness.png {code} CartesianProductConfig cpConfig = new CartesianProductConfig(crossProductSources); edgeManagerDescriptor.setUserPayload(cpConfig.toUserPayload(new TezConfiguration(conf))); {code} !Tez-CartesianProductSlowness.png! -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-21781) Vectorization: Incomplete vectorization of "or false" with CBO off
Gopal V created HIVE-21781: -- Summary: Vectorization: Incomplete vectorization of "or false" with CBO off Key: HIVE-21781 URL: https://issues.apache.org/jira/browse/HIVE-21781 Project: Hive Issue Type: Bug Components: Logical Optimizer Reporter: Gopal V {code} create temporary table foo (x int) ; insert into foo values(1),(2),(3),(4),(5); set hive.explain.user=false; explain vectorization detail select count(case when (x=1 or false) then 1 else 0 end ) from foo {code} {code} | Group By Operator | | aggregations: count(CASE WHEN (((x = 1) or false)) THEN (1) ELSE (0) END) | | Group By Vectorization: | | aggregators: VectorUDAFCount(IfExprLongScalarLongScalar(col 3:boolean, val 1, val 0)(children: VectorUDFAdaptor(((x = 1) or false))(children: LongColEqualLongScalar(col 0:int, val 1) -> 2:boolean) -> 3:boolean) -> 4:int) -> bigint | | className: VectorGroupByOperator | {code} The pass-through Calcite fixes this. {code} | OPTIMIZED SQL: SELECT COUNT(CASE WHEN `x` = 1 THEN 1 ELSE 0 END) AS `$f0` | | FROM `default`.`foo` {code} -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-21779) ACID: Delete deltas should not project the row struct
Gopal V created HIVE-21779: -- Summary: ACID: Delete deltas should not project the row struct Key: HIVE-21779 URL: https://issues.apache.org/jira/browse/HIVE-21779 Project: Hive Issue Type: Bug Components: Transactions Affects Versions: 3.1.1, 4.0.0 Reporter: Gopal V {code} if (useDecimal64ColumnVector) { this.batch = deleteDeltaReader.getSchema().createRowBatchV2(); } else { this.batch = deleteDeltaReader.getSchema().createRowBatch(); } {code} Is creating a fully wide row-batch, despite the fact that all row columns have to be NULL in a delete delta. createRowBatch() should follow includes and avoid over-allocating column vrbs. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-21768) JDBC: Strip the default union prefix for un-enclosed UNION queries
Gopal V created HIVE-21768: -- Summary: JDBC: Strip the default union prefix for un-enclosed UNION queries Key: HIVE-21768 URL: https://issues.apache.org/jira/browse/HIVE-21768 Project: Hive Issue Type: Bug Components: Logical Optimizer Affects Versions: 3.1.1, 4.0.0 Reporter: Gopal V {code} beeline> select 1 union all select 2; +--+ | _u1._c0 | +--+ | 2| | 1| +--+ {code} The _u1 is superfluous and harmful to result schema parsing. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-21742) Vectorization: CASE result type casting
Gopal V created HIVE-21742: -- Summary: Vectorization: CASE result type casting Key: HIVE-21742 URL: https://issues.apache.org/jira/browse/HIVE-21742 Project: Hive Issue Type: Bug Components: Logical Optimizer, Vectorization Affects Versions: 3.1.1 Reporter: Gopal V {code} create temporary table foo(q548284 int); insert into foo values(1),(2),(3),(4),(5),(6); select q548284, CASE WHEN ((q548284 = 1)) THEN (0.2) WHEN ((q548284 = 2)) THEN (0.4) WHEN ((q548284 = 3)) THEN (0.6) WHEN ((q548284 = 4)) THEN (0.8) WHEN ((q548284 = 5)) THEN (1) ELSE (null) END from foo order by q548284 limit 1; {code} Fails with {code} Caused by: java.lang.ClassCastException: org.apache.hadoop.hive.ql.exec.vector.LongColumnVector cannot be cast to org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector at org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector.setElement(DecimalColumnVector.java:130) at org.apache.hadoop.hive.ql.exec.vector.expressions.IfExprColumnNull.evaluate(IfExprColumnNull.java:101) {code} This gets fixed if the case return of (1) is turned into a (1.0). -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-21687) LLAP: LlapBaseInputFormat.getSplits(..., 0) should use LIMIT 0 query
Gopal V created HIVE-21687: -- Summary: LLAP: LlapBaseInputFormat.getSplits(..., 0) should use LIMIT 0 query Key: HIVE-21687 URL: https://issues.apache.org/jira/browse/HIVE-21687 Project: Hive Issue Type: Bug Reporter: Gopal V The 0 split query is useful in ensuring that the user has access to the tables and getting the result schema. There's a fast-path for Tableau for "LIMIT 0" queries which can be used for this without touching the get_splits UDTF. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-21646) Tez: Prevent TezTasks from escaping thread logging context
Gopal V created HIVE-21646: -- Summary: Tez: Prevent TezTasks from escaping thread logging context Key: HIVE-21646 URL: https://issues.apache.org/jira/browse/HIVE-21646 Project: Hive Issue Type: Bug Components: Tez Reporter: Gopal V If hive.exec.parallel is set to true to parallelize MoveTasks or StatsTasks, the Tez task does not benefit from a new thread and will lose all the thread context of the current query. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-21548) Statistics: String statistics computation is slower with vectorization enabled
Gopal V created HIVE-21548: -- Summary: Statistics: String statistics computation is slower with vectorization enabled Key: HIVE-21548 URL: https://issues.apache.org/jira/browse/HIVE-21548 Project: Hive Issue Type: Improvement Components: Statistics, Vectorization Reporter: Gopal V Attachments: string-stats-hll.png Computing the hashCode of a String for HLL is going byte[] -> Text -> String -> byte[] to get the Murmur3 hashCode of the byte[] region. !string-stats-hll.png! -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-21547) Temp Tables: Use stORC format for temporary tables
Gopal V created HIVE-21547: -- Summary: Temp Tables: Use stORC format for temporary tables Key: HIVE-21547 URL: https://issues.apache.org/jira/browse/HIVE-21547 Project: Hive Issue Type: Improvement Components: ORC Affects Versions: 4.0.0 Reporter: Gopal V Using st(reaming)ORC (hive.exec.orc.delta.streaming.optimizations.enabled=true) format has massive performance advantages when creating data-sets which will not be stored for long-term. The format is compatible with ORC for vectorization and other features, while being cheaper to write out to filesystem. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-21531) Vectorization: all NULL hashcodes are not computed using Murmur3
Gopal V created HIVE-21531: -- Summary: Vectorization: all NULL hashcodes are not computed using Murmur3 Key: HIVE-21531 URL: https://issues.apache.org/jira/browse/HIVE-21531 Project: Hive Issue Type: Bug Reporter: Gopal V The comments in Vectorized hash computation call out the MurmurHash implementation (the one using 0x5bd1e995), while the non-vectorized codepath calls out the Murmur3 one (using 0xcc9e2d51). The comments here are wrong {code} /** * Batch compute the hash codes for all the serialized keys. * * NOTE: MAJOR MAJOR ASSUMPTION: * We assume that HashCodeUtil.murmurHash produces the same result * as MurmurHash.hash with seed = 0 (the method used by ReduceSinkOperator for * UNIFORM distribution). */ protected void computeSerializedHashCodes() { int offset = 0; int keyLength; byte[] bytes = output.getData(); for (int i = 0; i < nonNullKeyCount; i++) { keyLength = serializedKeyLengths[i]; hashCodes[i] = Murmur3.hash32(bytes, offset, keyLength, 0); offset += keyLength; } } {code} but the wrong comment is followed in the Vector RS operator {code} System.arraycopy(nullKeyOutput.getData(), 0, nullBytes, 0, nullBytesLength); nullKeyHashCode = HashCodeUtil.calculateBytesHashCode(nullBytes, 0, nullBytesLength); {code} -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-21505) JDBC: Hive jdbc standalone does not have hadoop-common SASL servers
Gopal V created HIVE-21505: -- Summary: JDBC: Hive jdbc standalone does not have hadoop-common SASL servers Key: HIVE-21505 URL: https://issues.apache.org/jira/browse/HIVE-21505 Project: Hive Issue Type: Bug Reporter: Gopal V Either the detection code should use SASL client classes or the shading should include it. {code} Caused by: java.lang.NoClassDefFoundError: org/apache/hadoop/security/SaslRpcServer at org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge23.(HadoopThriftAuthBridge23.java:66) at org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge.getBridge(HadoopThriftAuthBridge.java:82) at org.apache.hive.service.auth.KerberosSaslHelper.getKerberosTransport(KerberosSaslHelper.java:55) at org.apache.hive.jdbc.HiveConnection.createBinaryTransport(HiveConnection.java:588) at org.apache.hive.jdbc.HiveConnection.openTransport(HiveConnection.java:337) at org.apache.hive.jdbc.HiveConnection.(HiveConnection.java:224) at org.apache.hive.jdbc.HiveDriver.connect(HiveDriver.java:107) at java.sql.DriverManager.getConnection(DriverManager.java:664) at java.sql.DriverManager.getConnection(DriverManager.java:270) at org.notmysock.jdbc.JDBCActor.call(JDBCActor.java:49) at org.notmysock.jdbc.JDBCActor.call(JDBCActor.java:15) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadP {code} -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-21447) Constraints: Adding a constraint twice takes 120+ seconds to fail
Gopal V created HIVE-21447: -- Summary: Constraints: Adding a constraint twice takes 120+ seconds to fail Key: HIVE-21447 URL: https://issues.apache.org/jira/browse/HIVE-21447 Project: Hive Issue Type: Bug Components: Metastore, Standalone Metastore Affects Versions: 4.0.0 Reporter: Gopal V It reconnects to the metastore 24 times before failing without a real reason. There's no useful error returned. {code} alter table customer add constraint ${DB}_c_fsd foreign key (c_first_sales_date_sk) references date_dim (d_date_sk) disable novalidate rely; INFO : Compiling command(queryId=hive_20190314061155_a060ca11-9547-4400-9564-9f0108c0e530): alter table customer add constraint tpcds_copy_orc_partitioned_1_c_fsd foreign key (c_first_sales_date_sk) references date_dim (d_date_sk) disable novalidate rely INFO : Semantic Analysis Completed (retrial = false) INFO : Returning Hive schema: Schema(fieldSchemas:null, properties:null) INFO : Completed compiling command(queryId=hive_20190314061155_a060ca11-9547-4400-9564-9f0108c0e530); Time taken: 0.021 seconds INFO : Executing command(queryId=hive_20190314061155_a060ca11-9547-4400-9564-9f0108c0e530): alter table customer add constraint tpcds_copy_orc_partitioned_1_c_fsd foreign key (c_first_sales_date_sk) references date_dim (d_date_sk) disable novalidate rely INFO : Starting task [Stage-0:DDL] in serial mode ERROR : FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. org.apache.thrift.TApplicationException: Internal error processing add_foreign_key INFO : Completed executing command(queryId=hive_20190314061155_a060ca11-9547-4400-9564-9f0108c0e530); Time taken: 122.712 seconds Error: Error while processing statement: FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. org.apache.thrift.TApplicationException: Internal error processing add_foreign_key (state=08S01,code=1) {code} {code} 2019-03-14T06:12:57,299 WARN [HiveServer2-Background-Pool: Thread-4295 (hive_20190314061155_a060ca11-9547-4400-9564-9f0108c0e530)] metastore.RetryingMetaStoreClient: MetaStoreClientlost connection. Attempting to reconnect (13 of 24) after 5s. addForeignKey org.apache.thrift.TApplicationException: Internal error processing add_foreign_key at org.apache.thrift.TApplicationException.read(TApplicationException.java:111) ~[hive-exec-3.1.0-SNAPSHOT.jar:3.1.0-SNAPSHOT] at org.apache.thrift.TServiceClient.receiveBase(TServiceClient.java:79) ~[hive-exec-3.1.0-SNAPSHOT.jar:3.1.0-SNAPSHOT] at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.recv_add_foreign_key(ThriftHiveMetastore.java:1738) ~[hive-exec-3.1.0-SNAPSHOT.jar:3.1.0-SNAPSHOT] at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.add_foreign_key(ThriftHiveMetastore.java:1725) ~[hive-exec-3.1.0-SNAPSHOT.jar:3.1.0-SNAPSHOT] at org.apache.hadoop.hive.metastore.HiveMetaStoreClient.addForeignKey(HiveMetaStoreClient.java:1033) ~[hive-exec-3.1.0-SNAPSHOT.jar:3.1.0-SNAPSHOT] at sun.reflect.GeneratedMethodAccessor323.invoke(Unknown Source) ~[?:?] at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) ~[?:1.8.0_181] at java.lang.reflect.Method.invoke(Method.java:498) ~[?:1.8.0_181] at org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.invoke(RetryingMetaStoreClient.java:212) ~[hive-exec-3.1.0-SNAPSHOT.jar:3.1.0-SNAPSHOT] at com.sun.proxy.$Proxy36.addForeignKey(Unknown Source) ~[?:?] at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) ~[?:1.8.0_181] at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) ~[?:1.8.0_181] at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) ~[?:1.8.0_181] at java.lang.reflect.Method.invoke(Method.java:498) ~[?:1.8.0_181] at org.apache.hadoop.hive.metastore.HiveMetaStoreClient$SynchronizedHandler.invoke(HiveMetaStoreClient.java:3010) ~[hive-exec-3.1.0-SNAPSHOT.jar:3.1.0-SNAPSHOT] at com.sun.proxy.$Proxy36.addForeignKey(Unknown Source) ~[?:?] at org.apache.hadoop.hive.ql.metadata.Hive.addForeignKey(Hive.java:5504) ~[hive-exec-3.1.0-SNAPSHOT.jar:3.1.0-SNAPSHOT] at org.apache.hadoop.hive.ql.exec.DDLTask.addConstraints(DDLTask.java:4080) ~[hive-exec-3.1.0-SNAPSHOT.jar:3.1.0-SNAPSHOT] {code} -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-21437) Vectorization: Decimal division with integer columns
Gopal V created HIVE-21437: -- Summary: Vectorization: Decimal division with integer columns Key: HIVE-21437 URL: https://issues.apache.org/jira/browse/HIVE-21437 Project: Hive Issue Type: Bug Components: Vectorization Reporter: Gopal V Vectorizer fails for {code} CREATE temporary TABLE `catalog_Sales`( `cs_quantity` int, `cs_wholesale_cost` decimal(7,2), `cs_list_price` decimal(7,2), `cs_sales_price` decimal(7,2), `cs_ext_discount_amt` decimal(7,2), `cs_ext_sales_price` decimal(7,2), `cs_ext_wholesale_cost` decimal(7,2), `cs_ext_list_price` decimal(7,2), `cs_ext_tax` decimal(7,2), `cs_coupon_amt` decimal(7,2), `cs_ext_ship_cost` decimal(7,2), `cs_net_paid` decimal(7,2), `cs_net_paid_inc_tax` decimal(7,2), `cs_net_paid_inc_ship` decimal(7,2), `cs_net_paid_inc_ship_tax` decimal(7,2), `cs_net_profit` decimal(7,2)) ; explain vectorization detail select maxcs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2) from catalog_sales; {code} {code} 'Map Vectorization:' 'enabled: true' 'enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true' 'inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat' 'notVectorizedReason: SELECT operator: Could not instantiate DecimalColDivideDecimalScalar with arguments arguments: [21, 20, 22], argument classes: [Integer, Integer, Integer], exception: java.lang.IllegalArgumentException: java.lang.ClassCastException@63b56be0 stack trace: sun.reflect.GeneratedConstructorAccessor.newInstance(Unknown Source), sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45), java.lang.reflect.Constructor.newInstance(Constructor.java:423), org.apache.hadoop.hive.ql.exec.vector.VectorizationContext.instantiateExpression(VectorizationContext.java:2088), org.apache.hadoop.hive.ql.optimizer.physical.Vectorizer.fixDecimalDataTypePhysicalVariations(Vectorizer.java:4662), org.apache.hadoop.hive.ql.optimizer.physical.Vectorizer.fixDecimalDataTypePhysicalVariations(Vectorizer.java:4602), org.apache.hadoop.hive.ql.optimizer.physical.Vectorizer.vectorizeSelectOperator(Vectorizer.java:4584), org.apache.hadoop.hive.ql.optimizer.physical.Vectorizer.validateAndVectorizeOperator(Vectorizer.java:5171), org.apache.hadoop.hive.ql.optimizer.physical.Vectorizer.doProcessChild(Vectorizer.java:923), org.apache.hadoop.hive.ql.optimizer.physical.Vectorizer.doProcessChildren(Vectorizer.java:809), org.apache.hadoop.hive.ql.optimizer.physical.Vectorizer.validateAndVectorizeOperatorTree(Vectorizer.java:776), org.apache.hadoop.hive.ql.optimizer.physical.Vectorizer.access$2400(Vectorizer.java:240), org.apache.hadoop.hive.ql.optimizer.physical.Vectorizer$VectorizationDispatcher.validateAndVectorizeMapOperators(Vectorizer.java:2038), org.apache.hadoop.hive.ql.optimizer.physical.Vectorizer$VectorizationDispatcher.validateAndVectorizeMapOperators(Vectorizer.java:1990), org.apache.hadoop.hive.ql.optimizer.physical.Vectorizer$VectorizationDispatcher.validateAndVectorizeMapWork(Vectorizer.java:1963), ...' 'vectorized: false' {code} -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-21417) JDBC: standalone jar relocates log4j interface classes
Gopal V created HIVE-21417: -- Summary: JDBC: standalone jar relocates log4j interface classes Key: HIVE-21417 URL: https://issues.apache.org/jira/browse/HIVE-21417 Project: Hive Issue Type: Bug Reporter: Gopal V The relocation of slf4j for ILoggerFactory breaks embedding in JVMs which have an slf4j impl locally. org/apache/hive/org/slf4j/ILoggerFactory Adding this jar to an existing slf4j env breaks. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-21400) Vectorization: LazyBinarySerializeWrite allocates Field() within the loop
Gopal V created HIVE-21400: -- Summary: Vectorization: LazyBinarySerializeWrite allocates Field() within the loop Key: HIVE-21400 URL: https://issues.apache.org/jira/browse/HIVE-21400 Project: Hive Issue Type: Bug Components: Vectorization Affects Versions: 4.0.0 Reporter: Gopal V GC thrash from an unexpected source in ReduceSinkOperator. {code} org.apache.hadoop.hive.serde2.lazybinary.fast.LazyBinarySerializeWrite.resetWithoutOutput(LazyBinarySerializeWrite.java:136) at org.apache.hadoop.hive.serde2.lazybinary.fast.LazyBinarySerializeWrite.reset(LazyBinarySerializeWrite.java:132) at org.apache.hadoop.hive.ql.exec.vector.reducesink.VectorReduceSinkUniformHashOperator.process(VectorReduceSinkUniformHashOperator.java:180) {code} GC space is getting thrashed by the {code} root = new Field(STRUCT); {code} for every row. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-21368) Vectorization: Unnecessary Decimal64 -> HiveDecimal conversion
Gopal V created HIVE-21368: -- Summary: Vectorization: Unnecessary Decimal64 -> HiveDecimal conversion Key: HIVE-21368 URL: https://issues.apache.org/jira/browse/HIVE-21368 Project: Hive Issue Type: Bug Reporter: Gopal V Joins projecting Decimal64 have a suspicious cast in the inner loop {code} ConvertDecimal64ToDecimal(col 14:decimal(7,2)/DECIMAL_64) -> 24:decimal(7,2)' {code} {code} create temporary table foo(x int , y decimal(7,2)); create temporary table bar(x int , y decimal(7,2)); set hive.explain.user=false; explain vectorization detail select sum(foo.y) from foo, bar where foo.x = bar.x; {code} {code} ' Map Join Operator' 'condition map:' ' Inner Join 0 to 1' 'keys:' ' 0 _col0 (type: int)' ' 1 _col0 (type: int)' 'Map Join Vectorization:' 'bigTableKeyColumnNums: [0]' 'bigTableRetainedColumnNums: [3]' 'bigTableValueColumnNums: [3]' 'bigTableValueExpressions: ConvertDecimal64ToDecimal(col 1:decimal(7,2)/DECIMAL_64) -> 3:decimal(7,2)' 'className: VectorMapJoinInnerBigOnlyLongOperator' 'native: true' 'nativeConditionsMet: hive.mapjoin.optimized.hashtable IS true, hive.vectorized.execution.mapjoin.native.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, One MapJoin Condition IS true, No nullsafe IS true, Small table vectorizes IS true, Fast Hash Table and No Hybrid Hash Join IS true' 'projectedOutputColumnNums: [3]' {code} -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-21344) CBO: Materialized view registry is not used for Calcite planner
Gopal V created HIVE-21344: -- Summary: CBO: Materialized view registry is not used for Calcite planner Key: HIVE-21344 URL: https://issues.apache.org/jira/browse/HIVE-21344 Project: Hive Issue Type: Bug Components: Materialized views Affects Versions: 4.0.0 Reporter: Gopal V Attachments: mv-get-from-remote.png {code} // This is not a rebuild, we retrieve all the materializations. In turn, we do not need // to force the materialization contents to be up-to-date, as this is not a rebuild, and // we apply the user parameters (HIVE_MATERIALIZED_VIEW_REWRITING_TIME_WINDOW) instead. materializations = db.getAllValidMaterializedViews(getTablesUsed(basePlan), false, getTxnMgr()); } {code} !mv-get-from-remote.png! -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-21343) CBO: CalcitePlanner debug logging is expensive and costly
Gopal V created HIVE-21343: -- Summary: CBO: CalcitePlanner debug logging is expensive and costly Key: HIVE-21343 URL: https://issues.apache.org/jira/browse/HIVE-21343 Project: Hive Issue Type: Bug Components: CBO Affects Versions: 4.0.0 Reporter: Gopal V Attachments: Reloptutil-toString.png {code} //Remove subquery LOG.debug("Plan before removing subquery:\n" + RelOptUtil.toString(calciteGenPlan)); calciteGenPlan = hepPlan(calciteGenPlan, false, mdProvider.getMetadataProvider(), null, new HiveSubQueryRemoveRule(conf)); LOG.debug("Plan just after removing subquery:\n" + RelOptUtil.toString(calciteGenPlan)); calciteGenPlan = HiveRelDecorrelator.decorrelateQuery(calciteGenPlan); LOG.debug("Plan after decorrelation:\n" + RelOptUtil.toString(calciteGenPlan)); {code} The LOG.debug() consumes more CPU than the actual planner steps. !Reloptutil-toString.png! -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-21340) CBO: Prune non-key columns feeding into a SemiJoin
Gopal V created HIVE-21340: -- Summary: CBO: Prune non-key columns feeding into a SemiJoin Key: HIVE-21340 URL: https://issues.apache.org/jira/browse/HIVE-21340 Project: Hive Issue Type: Bug Components: CBO Affects Versions: 4.0.0 Reporter: Gopal V {code} explain cbo with ss as (select count(1), ss_item_sk, ss_ticket_number from store_sales group by ss_item_sk, ss_ticket_number having count(1) > 1) select count(1) from item where i_item_sk IN (select ss_item_sk from ss); {code} Notice the {{HiveProject(ss_item_sk=[$0], ss_ticket_number=[$1], $f2=[$2])}} Only ss_item_sk is relevant for the HiveSemiJoin {code} CBO PLAN: HiveAggregate(group=[{}], agg#0=[count()]) HiveSemiJoin(condition=[=($0, $1)], joinType=[inner]) HiveProject(i_item_sk=[$0]) HiveFilter(condition=[IS NOT NULL($0)]) HiveTableScan(table=[[tpcds_copy_orc_partitioned_1, item]], table:alias=[item]) HiveProject(ss_item_sk=[$0], ss_ticket_number=[$1], $f2=[$2]) HiveFilter(condition=[>($2, 1)]) HiveAggregate(group=[{1, 8}], agg#0=[count()]) HiveFilter(condition=[IS NOT NULL($1)]) HiveTableScan(table=[[tpcds_copy_orc_partitioned_1, store_sales]], table:alias=[store_sales]) {code} -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-21339) LLAP: Cache hit also initializes an FS object
Gopal V created HIVE-21339: -- Summary: LLAP: Cache hit also initializes an FS object Key: HIVE-21339 URL: https://issues.apache.org/jira/browse/HIVE-21339 Project: Hive Issue Type: Bug Components: llap Affects Versions: 4.0.0 Reporter: Gopal V Attachments: llap-cache-fs-get.png https://github.com/apache/hive/blob/master/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java#L214 {code} // 1. Get file metadata from cache, or create the reader and read it. // Don't cache the filesystem object for now; Tez closes it and FS cache will fix all that fs = split.getPath().getFileSystem(jobConf); fileKey = determineFileId(fs, split, HiveConf.getBoolVar(daemonConf, ConfVars.LLAP_CACHE_ALLOW_SYNTHETIC_FILEID), HiveConf.getBoolVar(daemonConf, ConfVars.LLAP_CACHE_DEFAULT_FS_FILE_ID), !HiveConf.getBoolVar(daemonConf, ConfVars.LLAP_IO_USE_FILEID_PATH) ); {code} !llap-cache-fs-get.png! -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-21294) Vectorization: 1-reducer Shuffle can skip the object hash functions
Gopal V created HIVE-21294: -- Summary: Vectorization: 1-reducer Shuffle can skip the object hash functions Key: HIVE-21294 URL: https://issues.apache.org/jira/browse/HIVE-21294 Project: Hive Issue Type: Bug Components: Vectorization Reporter: Gopal V VectorObjectSinkHashOperator can skip the object hashing entirely if the reducer count = 1. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-21268) REPL: Repl dump can output - Database, Table, Dir, last_repl_id
Gopal V created HIVE-21268: -- Summary: REPL: Repl dump can output - Database, Table, Dir, last_repl_id Key: HIVE-21268 URL: https://issues.apache.org/jira/browse/HIVE-21268 Project: Hive Issue Type: Improvement Reporter: Gopal V {code} INFO : Completed executing command(queryId=root_20190214061031_639e3a52-5c62-40be-a3cd-3e0b18b7b41d); Time taken: 0.374 seconds INFO : OK ++---+ | dump_dir | last_repl_id | ++---+ | /user/root/repl/a74389d0-7cde-4cf4-aa40-3079a98b80a8 | 1104594 | ++---+ 1 row selected (0.445 seconds) {code} is somewhat hard to associate back to the table name. The logs a couple of lines above actually print the operation detail. {code} INFO : REPL::TABLE_DUMP: {"dbName":"tpcds_bin_partitioned_orc_1000","tableName":"item","tableType":"MANAGED_TABLE","tablesDumpProgress":"1/38","dumpTime":1550124632} {code} -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-21233) ACID: Fix build warnings from HIVE-20699
Gopal V created HIVE-21233: -- Summary: ACID: Fix build warnings from HIVE-20699 Key: HIVE-21233 URL: https://issues.apache.org/jira/browse/HIVE-21233 Project: Hive Issue Type: Bug Components: Transactions Affects Versions: 4.0.0 Reporter: Gopal V {code} Lines that start with ? in the ASF License report indicate files that do not have an Apache license header: !? /data/hiveptest/working/yetus_PreCommit-HIVE-Build-15992/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFValidateAcidSortOrder.java {code} -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-21232) LLAP: Add a cache-miss friendly split affinity provider
Gopal V created HIVE-21232: -- Summary: LLAP: Add a cache-miss friendly split affinity provider Key: HIVE-21232 URL: https://issues.apache.org/jira/browse/HIVE-21232 Project: Hive Issue Type: Bug Reporter: Gopal V If one of the LLAP nodes have data-locality, preferring that over another does have advantages for the first query or a more general cache-miss. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-21229) Tez: Provide a Node-Label split affinity provider
Gopal V created HIVE-21229: -- Summary: Tez: Provide a Node-Label split affinity provider Key: HIVE-21229 URL: https://issues.apache.org/jira/browse/HIVE-21229 Project: Hive Issue Type: Bug Reporter: Gopal V When the queue has node-labels, the delays involved in waiting for node locality are a net-loss to the query. Asking for locality in a node which is not available to the current queue is slowing down YARN task allocations. Also specifically, for the 3-replica HDFS case, picking the replica that belongs within the node-label is more efficient than picking a non-available node & falling back to rack_local instead. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-21225) ACID: getAcidState() should cache a recursive dir listing locally
Gopal V created HIVE-21225: -- Summary: ACID: getAcidState() should cache a recursive dir listing locally Key: HIVE-21225 URL: https://issues.apache.org/jira/browse/HIVE-21225 Project: Hive Issue Type: Bug Components: Transactions Reporter: Gopal V Currently getAcidState() makes 3 calls into the FS api which could be answered by making a single recursive listDir call and reusing the same data to check for isRawFormat() and isValidBase(). All delta operations for a single partition can go against a single listed directory snapshot instead of interacting with the NameNode or ObjectStore within the inner loop. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-21012) CBO: CASE + CONCAT expr is not converted to optimized SQL
Gopal V created HIVE-21012: -- Summary: CBO: CASE + CONCAT expr is not converted to optimized SQL Key: HIVE-21012 URL: https://issues.apache.org/jira/browse/HIVE-21012 Project: Hive Issue Type: Bug Components: CBO Affects Versions: 4.0.0 Reporter: Gopal V {code} create temporary table t3 (receipt_required_flag string, inspection_required_flag string); explain extended select CASE WHEN concat(concat(coalesce(t3.receipt_required_flag, 'N'), '~'), coalesce(t3.inspection_required_flag, 'N')) = 'N~N' THEN '2-Way Match' WHEN concat(concat(coalesce(t3.receipt_required_flag, 'N'), '~'), coalesce(t3.inspection_required_flag, 'N')) = 'Y~N' THEN '3-Way Match' WHEN concat(concat(coalesce(t3.receipt_required_flag, 'N'), '~'), coalesce(t3.inspection_required_flag, 'N')) = 'Y~Y' THEN '4-Way Match' END from t3; {code} does not print an optimized SQL because {code} 2018-12-05T22:52:26,193 WARN [HiveServer2-Background-Pool: Thread-2100] parse.CalcitePlanner: Rel2SQL Rewrite threw error java.lang.UnsupportedOperationException: class org.apache.calcite.sql.SqlSyntax$6: SPECIAL at org.apache.calcite.util.Util.needToImplement(Util.java:927) ~[calcite-core-1.16.0.3.1.0.0-SNAPSHOT.jar:1.16.0.3.1.0.0-SNAPSHOT] at org.apache.calcite.sql.SqlSyntax$6.unparse(SqlSyntax.java:116) ~[calcite-core-1.16.0.3.1.0.0-SNAPSHOT.jar:1.16.0.3.1.0.0-SNAPSHOT] at org.apache.calcite.sql.SqlOperator.unparse(SqlOperator.java:332) ~[calcite-core-1.16.0.3.1.0.0-SNAPSHOT.jar:1.16.0.3.1.0.0-SNAPSHOT] at org.apache.calcite.sql.SqlDialect.unparseCall(SqlDialect.java:333) ~[calcite-core-1.16.0.3.1.0.0-SNAPSHOT.jar:1.16.0.3.1.0.0-SNAPSHOT] at org.apache.calcite.sql.SqlCall.unparse(SqlCall.java:103) ~[calcite-core-1.16.0.3.1.0.0-SNAPSHOT.jar:1.16.0.3.1.0.0-SNAPSHOT] at org.apache.calcite.sql.SqlUtil.unparseBinarySyntax(SqlUtil.java:323) ~[calcite-core-1.16.0.3.1.0.0-SNAPSHOT.jar:1.16.0.3.1.0.0-SNAPSHOT] at org.apache.calcite.sql.SqlSyntax$3.unparse(SqlSyntax.java:65) ~[calcite-core-1.16.0.3.1.0.0-SNAPSHOT.jar:1.16.0.3.1.0.0-SNAPSHOT] at org.apache.calcite.sql.SqlOperator.unparse(SqlOperator.java:332) ~[calcite-core-1.16.0.3.1.0.0-SNAPSHOT.jar:1.16.0.3.1.0.0-SNAPSHOT] at org.apache.calcite.sql.SqlDialect.unparseCall(SqlDialect.java:333) ~[calcite-core-1.16.0.3.1.0.0-SNAPSHOT.jar:1.16.0.3.1.0.0-SNAPSHOT] at org.apache.calcite.sql.SqlCall.unparse(SqlCall.java:103) ~[calcite-core-1.16.0.3.1.0.0-SNAPSHOT.jar:1.16.0.3.1.0.0-SNAPSHOT] at org.apache.calcite.sql.fun.SqlCaseOperator.unparse(SqlCaseOperator.java:306) ~[calcite-core-1.16.0.3.1.0.0-SNAPSHOT.jar:1.16.0.3.1.0.0-SNAPSHOT] at org.apache.calcite.sql.SqlDialect.unparseCall(SqlDialect.java:333) ~[calcite-core-1.16.0.3.1.0.0-SNAPSHOT.jar:1.16.0.3.1.0.0-SNAPSHOT] at org.apache.calcite.sql.SqlCall.unparse(SqlCall.java:103) ~[calcite-core-1.16.0.3.1.0.0-SNAPSHOT.jar:1.16.0.3.1.0.0-SNAPSHOT] at org.apache.calcite.sql.SqlAsOperator.unparse(SqlAsOperator.java:76) ~[calcite-core-1.16.0.3.1.0.0-SNAPSHOT.jar:1.16.0.3.1.0.0-SNAPSHOT] at org.apache.calcite.sql.SqlDialect.unparseCall(SqlDialect.java:333) ~[calcite-core-1.16.0.3.1.0.0-SNAPSHOT.jar:1.16.0.3.1.0.0-SNAPSHOT] at org.apache.calcite.sql.SqlCall.unparse(SqlCall.java:103) ~[calcite-core-1.16.0.3.1.0.0-SNAPSHOT.jar:1.16.0.3.1.0.0-SNAPSHOT] at org.apache.calcite.sql.SqlNodeList.commaList(SqlNodeList.java:121) ~[calcite-core-1.16.0.3.1.0.0-SNAPSHOT.jar:1.16.0.3.1.0.0-SNAPSHOT] at org.apache.calcite.sql.SqlOperator.unparseListClause(SqlOperator.java:349) ~[calcite-core-1.16.0.3.1.0.0-SNAPSHOT.jar:1.16.0.3.1.0.0-SNAPSHOT] at org.apache.calcite.sql.SqlOperator.unparseListClause(SqlOperator.java:338) ~[calcite-core-1.16.0.3.1.0.0-SNAPSHOT.jar:1.16.0.3.1.0.0-SNAPSHOT] at org.apache.calcite.sql.SqlSelectOperator.unparse(SqlSelectOperator.java:152) ~[calcite-core-1.16.0.3.1.0.0-SNAPSHOT.jar:1.16.0.3.1.0.0-SNAPSHOT] at org.apache.calcite.sql.SqlSelect.unparse(SqlSelect.java:240) ~[calcite-core-1.16.0.3.1.0.0-SNAPSHOT.jar:1.16.0.3.1.0.0-SNAPSHOT] at org.apache.calcite.sql.SqlNode.toSqlString(SqlNode.java:152) ~[calcite-core-1.16.0.3.1.0.0-SNAPSHOT.jar:1.16.0.3.1.0.0-SNAPSHOT] at org.apache.calcite.sql.SqlNode.toSqlString(SqlNode.java:158) ~[calcite-core-1.16.0.3.1.0.0-SNAPSHOT.jar:1.16.0.3.1.0.0-SNAPSHOT] at org.apache.hadoop.hive.ql.parse.CalcitePlanner.getOptimizedSql(CalcitePlanner.java:1477) ~[hive-exec-3.1.0-SNAPSHOT.jar:3.1.0-SNAPSHOT] {code} -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-21005) LLAP: Reading more stripes per-split leaks ZlibCodecs
Gopal V created HIVE-21005: -- Summary: LLAP: Reading more stripes per-split leaks ZlibCodecs Key: HIVE-21005 URL: https://issues.apache.org/jira/browse/HIVE-21005 Project: Hive Issue Type: Bug Reporter: Gopal V OrcEncodedDataReader - calls ensureDataReader in a loop, overwriting itself {code} for (int stripeIxMod = 0; stripeIxMod < stripeRgs.length; ++stripeIxMod) { // 6.2. Ensure we have stripe metadata. We might have read it before for RG filtering. if (stripeMetadatas != null) { stripeMetadata = stripeMetadatas.get(stripeIxMod); } else { ... ensureDataReader(); ... } {code} {code} private void ensureDataReader() throws IOException { ... stripeReader = orcReader.encodedReader( fileKey, dw, dw, useObjectPools ? POOL_FACTORY : null, trace, useCodecPool, cacheTag); {code} creates new encodedReader without closing previous stripe's encoded reader. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20999) LLAP IO: MutableQuantiles is contended heavily
Gopal V created HIVE-20999: -- Summary: LLAP IO: MutableQuantiles is contended heavily Key: HIVE-20999 URL: https://issues.apache.org/jira/browse/HIVE-20999 Project: Hive Issue Type: Bug Components: llap Affects Versions: 3.1.1 Reporter: Gopal V MutableQuantiles::add() is synchronized across all threads. {code} IO-Elevator-Thread-0 [DAEMON] State: BLOCKED CPU usage on sample: 316ms org.apache.hadoop.metrics2.lib.MutableQuantiles.add(long) MutableQuantiles.java:133 org.apache.hadoop.hive.llap.metrics.LlapDaemonIOMetrics.addDecodeBatchTime(long) LlapDaemonIOMetrics.java:98 org.apache.hadoop.hive.llap.io.decode.EncodedDataConsumer.consumeData(EncodedColumnBatch) EncodedDataConsumer.java:89 org.apache.hadoop.hive.llap.io.decode.EncodedDataConsumer.consumeData(Object) EncodedDataConsumer.java:34 org.apache.hadoop.hive.ql.io.orc.encoded.EncodedReaderImpl.readEncodedColumns(int, StripeInformation, OrcProto$RowIndex[], List, List, boolean[], boolean[], Consumer) EncodedReaderImpl.java:530 org.apache.hadoop.hive.llap.io.encoded.OrcEncodedDataReader.performDataRead() OrcEncodedDataReader.java:407 {code} -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20989) JDBC: The GetOperationStatus + log can block query progress via sleep()
Gopal V created HIVE-20989: -- Summary: JDBC: The GetOperationStatus + log can block query progress via sleep() Key: HIVE-20989 URL: https://issues.apache.org/jira/browse/HIVE-20989 Project: Hive Issue Type: Bug Reporter: Gopal V There is an exponential sleep operation inside the CLIService which can end up adding tens of seconds to a query which has already completed. {code} "HiveServer2-Handler-Pool: Thread-9373" #9373 prio=5 os_prio=0 tid=0x7f4d5e72d800 nid=0xb634a waiting on condition [0x7f28d06a5000] java.lang.Thread.State: TIMED_WAITING (sleeping) at java.lang.Thread.sleep(Native Method) at org.apache.hive.service.cli.CLIService.progressUpdateLog(CLIService.java:506) at org.apache.hive.service.cli.CLIService.getOperationStatus(CLIService.java:480) at org.apache.hive.service.cli.thrift.ThriftCLIService.GetOperationStatus(ThriftCLIService.java:695) at org.apache.hive.service.rpc.thrift.TCLIService$Processor$GetOperationStatus.getResult(TCLIService.java:1757) at org.apache.hive.service.rpc.thrift.TCLIService$Processor$GetOperationStatus.getResult(TCLIService.java:1742) at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:39) at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39) at org.apache.hive.service.auth.TSetIpAddressProcessor.process(TSetIpAddressProcessor.java:56) at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:286) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) {code} The sleep loop is on the server side. {code} private static final long PROGRESS_MAX_WAIT_NS = 30 * 10l; private JobProgressUpdate progressUpdateLog(boolean isProgressLogRequested, Operation operation, HiveConf conf) { ... long startTime = System.nanoTime(); int timeOutMs = 8; try { while (sessionState.getProgressMonitor() == null && !operation.isDone()) { long remainingMs = (PROGRESS_MAX_WAIT_NS - (System.nanoTime() - startTime)) / 100l; if (remainingMs <= 0) { LOG.debug("timed out and hence returning progress log as NULL"); return new JobProgressUpdate(ProgressMonitor.NULL); } Thread.sleep(Math.min(remainingMs, timeOutMs)); timeOutMs <<= 1; } {code} After about 16 seconds of execution of the query, the timeOutMs is 16384 ms, which means the next sleep cycle is for min(30 - 17, 16) = 13. If the query finishes on the 17th second, the JDBC server will only respond after the 30th second when it will check for operation.isDone() and return. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20983) Vectorization: Scale up small hashtables, when collisions are detected
Gopal V created HIVE-20983: -- Summary: Vectorization: Scale up small hashtables, when collisions are detected Key: HIVE-20983 URL: https://issues.apache.org/jira/browse/HIVE-20983 Project: Hive Issue Type: Bug Reporter: Gopal V Hive's hashtable estimates are getting better with HyperLogLog stats in place, but an accurate estimate does not always result in a low number of collisions. The hashtables which contain a very small number of items tend to lose their O(1) lookup performance where there are collisions. Since collisions are easy to detect within the fast hashtable implementation, a rehashing to a higher size will help these small hashtables avoid collisions and go back to O(1) perf. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20973) Optimizer: Reduce de-dup changes the hash-function of a reducer edge
Gopal V created HIVE-20973: -- Summary: Optimizer: Reduce de-dup changes the hash-function of a reducer edge Key: HIVE-20973 URL: https://issues.apache.org/jira/browse/HIVE-20973 Project: Hive Issue Type: Bug Reporter: Gopal V {code} private static void propagateMaxNumReducers(ReduceSinkJoinDeDuplicateProcCtx dedupCtx, ReduceSinkOperator rsOp, int maxNumReducers) throws SemanticException { if (rsOp == null) { // Bail out return; } if (rsOp.getChildOperators().get(0) instanceof MapJoinOperator || rsOp.getChildOperators().get(0) instanceof CommonMergeJoinOperator) { for (Operator p : rsOp.getChildOperators().get(0).getParentOperators()) { ReduceSinkOperator pRSOp = (ReduceSinkOperator) p; pRSOp.getConf().setReducerTraits(EnumSet.of(ReducerTraits.FIXED)); pRSOp.getConf().setNumReducers(maxNumReducers); LOG.debug("Set {} to FIXED parallelism: {}", pRSOp, maxNumReducers); if (pRSOp.getConf().isForwarding()) { ReduceSinkOperator newRSOp = CorrelationUtilities.findFirstPossibleParent( pRSOp, ReduceSinkOperator.class, dedupCtx.trustScript()); propagateMaxNumReducers(dedupCtx, newRSOp, maxNumReducers); } } {code} FIXED used to mean AUTOPARALLEL=false, but now FIXED means a different hash function from UNIFORM. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20951) LLAP: Set Xms to 50% always
Gopal V created HIVE-20951: -- Summary: LLAP: Set Xms to 50% always Key: HIVE-20951 URL: https://issues.apache.org/jira/browse/HIVE-20951 Project: Hive Issue Type: Bug Components: llap Affects Versions: 3.1.1, 4.0.0 Reporter: Gopal V The lack of GC pauses is killing LLAP containers whenever the significant amount of memory is consumed by the off-heap structures which aren't cleaned up automatically until the GC runs. There's a java.nio.DirectByteBuffer.Deallocator which runs when the Direct buffers are garbage collected, which actually does the cleanup of the underlying off-heap buffers. The lack of Garbage collection activity for several hours while responding to queries triggers a build-up of these off-heap structures which end up forcing YARN to kill the process instead. It is better to hit a GC pause occasionally rather than to lose a node every few hours. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20888) TxnHandler: sort() called on immutable lists
Gopal V created HIVE-20888: -- Summary: TxnHandler: sort() called on immutable lists Key: HIVE-20888 URL: https://issues.apache.org/jira/browse/HIVE-20888 Project: Hive Issue Type: Bug Reporter: Gopal V {code} } else { assert (!rqst.isSetSrcTxnToWriteIdList()); assert (rqst.isSetTxnIds()); txnIds = rqst.getTxnIds(); } Collections.sort(txnIds); //easier to read logs and for assumption done in replication flow {code} when the input comes from {code} @Override public long allocateTableWriteId(long txnId, String dbName, String tableName) throws TException { return allocateTableWriteIdsBatch(Collections.singletonList(txnId), dbName, tableName).get(0).getWriteId(); } {code} {code} java.lang.UnsupportedOperationException: null at java.util.AbstractList.set(AbstractList.java:132) ~[?:1.8.0] at java.util.AbstractList$ListItr.set(AbstractList.java:426) ~[?:1.8.0] at java.util.Collections.sort(Collections.java:170) ~[?:1.8.0] at org.apache.hadoop.hive.metastore.txn.TxnHandler.allocateTableWriteIds(TxnHandler.java:1523) ~[hive-standalone-metastore-server-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT] at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.allocate_table_write_ids(HiveMetaStore.java:7349) ~[hive-standalone-metastore-server-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT] {code} -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20887) Tests: openjdk 8 has a bug that prevents surefire from working
Gopal V created HIVE-20887: -- Summary: Tests: openjdk 8 has a bug that prevents surefire from working Key: HIVE-20887 URL: https://issues.apache.org/jira/browse/HIVE-20887 Project: Hive Issue Type: Bug Reporter: Gopal V It looks like the problem is https://bugs.openjdk.java.net/browse/JDK-8030046. It looks like: {code:bash} [ERROR] Caused by: org.apache.maven.surefire.booter.SurefireBooterForkException: The forked VM terminated without properly saying goodbye. VM crash or System.exit called? {code} The surefire-reports/*.dumpstream looks like: {code:bash} Error: Could not find or load main class org.apache.maven.surefire.booter.ForkedBooter {code} and we can work around the problem by changing the surefire configuration: {code:bash} + false {code} -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20890) ACID: Allow whole table ReadLocks to skip all partition locks
Gopal V created HIVE-20890: -- Summary: ACID: Allow whole table ReadLocks to skip all partition locks Key: HIVE-20890 URL: https://issues.apache.org/jira/browse/HIVE-20890 Project: Hive Issue Type: Bug Components: Transactions Reporter: Gopal V HIVE-19369 proposes adding a EXCL_WRITE lock which does not wait for any SHARED_READ locks for insert operations - in the presence of that lock, the insert overwrite no longer takes an exclusive lock. The only exclusive operation will be a schema change or drop table, which should take an exclusive lock on the entire table directly. {code} explain locks select * from tpcds_bin_partitioned_orc_1000.store_sales where ss_sold_date_sk=2452626 ++ | Explain | ++ | LOCK INFORMATION: | | tpcds_bin_partitioned_orc_1000.store_sales -> SHARED_READ | | tpcds_bin_partitioned_orc_1000.store_sales.ss_sold_date_sk=2452626 -> SHARED_READ | ++ {code} So the per-partition SHARED_READ locks are no longer necessary, if the lock builder already includes the table-wide SHARED_READ locks. The removal of entire partitions is the only part which needs to be taken care of within this semantics as row-removal instead of directory removal (i.e "drop partition" -> "truncate partition" and have the truncation trigger a whole directory cleaner, so that the partition disappears when there are 0 rows left). -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20854) Sensible Defaults: Hive's Zookeeper heartbeat interval is 20 minutes, change to 2
Gopal V created HIVE-20854: -- Summary: Sensible Defaults: Hive's Zookeeper heartbeat interval is 20 minutes, change to 2 Key: HIVE-20854 URL: https://issues.apache.org/jira/browse/HIVE-20854 Project: Hive Issue Type: Bug Reporter: Gopal V {code} HIVE_ZOOKEEPER_SESSION_TIMEOUT("hive.zookeeper.session.timeout", "120ms", new TimeValidator(TimeUnit.MILLISECONDS), "ZooKeeper client's session timeout (in milliseconds). The client is disconnected, and as a result, all locks released, \n" + "if a heartbeat is not sent in the timeout."), {code} That's 1,200,000ms which is too long for all practical purposes - a 20 minute outage in case a node has a failure is too long. That is too long for the JDBC load-balancing, LLAP failure tolerance and the lock manager expiry. Change to 2 minutes, as a sensible default -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20802) Statistics: Provide per-column statistics expanded for "explain formatted"
Gopal V created HIVE-20802: -- Summary: Statistics: Provide per-column statistics expanded for "explain formatted" Key: HIVE-20802 URL: https://issues.apache.org/jira/browse/HIVE-20802 Project: Hive Issue Type: Bug Reporter: Gopal V Debugging the internal per-column statistics of Hive is challenging, particularly when the estimations are approximate (hive.stats.estimate=true). Adding an extra columnStatistics field for all operators in "explain formatted" will allow tools to analyze the internal statistics transforms done by each operator without having to refer to debug logs. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20801) ACID: Allow DbTxnManager to ignore non-ACID table locking
Gopal V created HIVE-20801: -- Summary: ACID: Allow DbTxnManager to ignore non-ACID table locking Key: HIVE-20801 URL: https://issues.apache.org/jira/browse/HIVE-20801 Project: Hive Issue Type: Bug Reporter: Gopal V Enabling ACIDv1 on a cluster produces a central locking bottleneck for all table types, which is not always the intention. The Hive locking for non-acid tables are advisory (i.e a client can write/read without locking), which means that the implementation does not offer strong consistency despite the lock manager consuming resources centrally. Disabling this lock acquisition would improve the performance of non-ACID tables co-existing with a globally configured DbTxnManager implementation. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20764) Tez: ReduceRecordProcessor::init() can avoid doing .waitForAllInputsReady()
Gopal V created HIVE-20764: -- Summary: Tez: ReduceRecordProcessor::init() can avoid doing .waitForAllInputsReady() Key: HIVE-20764 URL: https://issues.apache.org/jira/browse/HIVE-20764 Project: Hive Issue Type: Bug Reporter: Gopal V The ReduceRecordProcessor waits for all inputs to be ready before starting to initialize the input operators. This is not necessary for the unsorted shuffle cases, as just having one input (the hash side) entirely ready is sufficient to start doing useful work with the operators (i.e build the hashtable). {code} if (shuffleInputs != null) { l4j.info("Waiting for ShuffleInputs to become ready"); processorContext.waitForAllInputsReady(new ArrayList(shuffleInputs)); } {code} -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20758) Constraints: Show create table does not show constraints
Gopal V created HIVE-20758: -- Summary: Constraints: Show create table does not show constraints Key: HIVE-20758 URL: https://issues.apache.org/jira/browse/HIVE-20758 Project: Hive Issue Type: Bug Reporter: Gopal V Even though the desc formatted shows the constraints, the show create table does not {code} | # Primary Key | NULL | NULL | | Table: | tpcds_bin_partitioned_orc_1.inventory | NULL | | Constraint Name: | pk_in | NULL | | Column Names: | inv_date_sk | inv_item_sk| || NULL | NULL | | # Foreign Keys | NULL | NULL | | Table: | tpcds_bin_partitioned_orc_1.inventory | NULL | | Constraint Name: | inv_d | NULL | | Parent Column Name:tpcds_bin_partitioned_orc_1.date_dim.d_date_sk | Column Name:inv_date_sk| Key Sequence:1 | || NULL | NULL | | Constraint Name: | inv_i | NULL | | Parent Column Name:tpcds_bin_partitioned_orc_1.item.i_item_sk | Column Name:inv_item_sk| Key Sequence:1 | || NULL | NULL | | Constraint Name: | inv_w | NULL | | Parent Column Name:tpcds_bin_partitioned_orc_1.warehouse.w_warehouse_sk | Column Name:inv_warehouse_sk | Key Sequence:1 | || NULL | NULL | {code} But {code} ++ | createtab_stmt | ++ | CREATE TABLE `inventory`( | | `inv_item_sk` bigint,| | `inv_warehouse_sk` bigint, | | `inv_quantity_on_hand` int, | | `inv_date_sk` bigint)| | ROW FORMAT SERDE | | 'org.apache.hadoop.hive.ql.io.orc.OrcSerde' | | STORED AS INPUTFORMAT | | 'org.apache.hadoop.hive.ql.io.orc.OrcInputFormat' | | OUTPUTFORMAT | | 'org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat' | | LOCATION | | 'hdfs:///warehouse/tablespace/managed/hive/tpcds_bin_partitioned_orc_1.db/inventory' | | TBLPROPERTIES (| | 'bucketing_version'='2', | | 'transactional'='true', | | 'transactional_properties'='default',| | 'transient_lastDdlTime'='1539710410')| ++ {code} -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20749) Constraints: Null scan optimizer does not remove entire TableScan
Gopal V created HIVE-20749: -- Summary: Constraints: Null scan optimizer does not remove entire TableScan Key: HIVE-20749 URL: https://issues.apache.org/jira/browse/HIVE-20749 Project: Hive Issue Type: Bug Components: CBO, Logical Optimizer Reporter: Gopal V With PK constraints on store table, the following query fails to remove the entire join (and cut the query short). {code} explain select count(1) from store_sales, store where ss_store_sk = s_store_sk and s_store_sk is null and ss_sold_date_sk between 2450816 and 2450816+365; {code} Calcite generates the following optimized query, but the NullScan optimizer does not kick in for the query. {code} SELECT COUNT(*) AS `$f0` FROM (SELECT * FROM (SELECT 0 AS `DUMMY` FROM `tpcds_bin_partitioned_orc_1`.`store_sales`) AS `t` WHERE FALSE) AS `t0`, (SELECT * FROM (SELECT 0 AS `DUMMY` FROM `tpcds_bin_partitioned_orc_1`.`store`) AS `t1` WHERE FALSE) AS `t2` {code} -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20739) CBO: Rewrite INTERSECTS queries as EXISTS semi-joins
Gopal V created HIVE-20739: -- Summary: CBO: Rewrite INTERSECTS queries as EXISTS semi-joins Key: HIVE-20739 URL: https://issues.apache.org/jira/browse/HIVE-20739 Project: Hive Issue Type: Improvement Reporter: Gopal V INTERSECTS clause currently materializes both sides before removing entries from each other. This means that if the intersects has filters on one side which apply to the other side, it is not transitively pushed to the other side. Here's the snippet from the TPC-DS Query8 where that is visible. {code} SELECT substr(ca_zip,1,5) ca_zip FROM customer_address WHERE substr(ca_zip,1,5) IN ( '89436', /* ... */ ','15492')) intersect (select ca_zip from (SELECT substr(ca_zip,1,5) ca_zip,count(*) cnt FROM customer_address, customer WHERE ca_address_sk = c_current_addr_sk and c_preferred_cust_flag='Y' group by ca_zip having count(*) > 10)A1))A2 {code} https://github.com/hortonworks/hive-testbench/blob/hdp3/sample-queries-tpcds/query8.sql#L92 where the entire join output of (customer_address x customer) is produced where the transitive filter inference could be applied to push the filter to the customer_address in the 2nd set in intersection. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20624) Vectorization: Fast Hash table should not double after certain size, instead should grow
Gopal V created HIVE-20624: -- Summary: Vectorization: Fast Hash table should not double after certain size, instead should grow Key: HIVE-20624 URL: https://issues.apache.org/jira/browse/HIVE-20624 Project: Hive Issue Type: Bug Components: Vectorization Reporter: Gopal V The reason to use Power of two is to simplify the inner loop for the hash function, but this is a significant memory issue when dealing with somewhat larger hashtables like the customer or customer_address hashtables in TPC-DS. This doubling is pretty bad when the hashtable load-factor is 0.75 and the expected key count is 65M (for customer/customer_address) {code} long worstCaseNeededSlots = 1L << DoubleMath.log2(numRows / hashTableLoadFactor, RoundingMode.UP); {code} That estimate is actually matching the actual impl, but after acquiring 65M items in a single array, the rehashing will require a temporary growth to 65+128 while the rehash is in progress, all to fit exactly 65 back into it. Fixing the estimate to match the implementation produced a number of regressions in query runtimes, though the part that needs fixing is the doubling implementation. The obvious solution is to add 4M more everytime and use a modulo function or the Lemire's multiply + shift operation[1], but more on that in comments. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20623) Shared work: Extend sharing of map-join cache entries in LLAP
Gopal V created HIVE-20623: -- Summary: Shared work: Extend sharing of map-join cache entries in LLAP Key: HIVE-20623 URL: https://issues.apache.org/jira/browse/HIVE-20623 Project: Hive Issue Type: Improvement Components: llap, Logical Optimizer Reporter: Gopal V Attachments: hash-shared-work.svg For a query like this {code} with all_sales as ( select ss_customer_sk as customer_sk, ss_ext_list_price-ss_ext_discount_amt as ext_price from store_sales UNION ALL select ws_bill_customer_sk as customer_sk, ws_ext_list_price-ws_ext_discount_amt as ext_price from web_sales UNION ALL select cs_bill_customer_sk as customer_sk, cs_ext_sales_price - cs_ext_discount_amt as ext_price from catalog_sales) select sum(ext_price) total_price, c_customer_id from all_sales, customer where customer_sk = c_customer_sk group by c_customer_id order by total_price desc limit 100; {code} The hashtable used for all 3 joins are identical, which is loaded 3x times in the same LLAP instance because they are named. {code} cacheKey = "HASH_MAP_" + this.getOperatorId() + "_container"; {code} in the cache. If those are identical in nature (i.e vectorization, hashtable type etc), then the duplication is just wasted CPU, memory and network - using the cache name for hashtables which will be identical in layout would be extremely useful. In cases where the join is pushed through a UNION, those are identical. This optimization can only be done without concern for accidental delays when the same upstream task is generating all of these hashtables, which is what is achieved by the shared scan optimizer already. In case the shared work is not present, this has potential downsides - in case two customer broadcasts were sourced from "Map 1" and "Map 2", the Map 1 builder will block the other task from reading from Map 2, even though Map 2 might have started after, but finished ahead of Map 1. So this specific optimization can always be considered for cases where the shared work unifies the operator tree and the parents of all the RS entries involved are same (& the RS layout is the same). -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20622) Backwards Compat: Support non-zero padded date and timestamp entries
Gopal V created HIVE-20622: -- Summary: Backwards Compat: Support non-zero padded date and timestamp entries Key: HIVE-20622 URL: https://issues.apache.org/jira/browse/HIVE-20622 Project: Hive Issue Type: Bug Reporter: Gopal V The Date spec in Hive says https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Types#LanguageManualTypes-date {noformat} DATE values describe a particular year/month/day, in the form -ÂMM-ÂDD. For example, DATE '2013-Â01-Â01' {noformat} Even though the Hive date & timestamp spec are strict on their format specification, Hive has previously supported " -[m]m-[d]d" formatted dates and it is better to expand the spec rather than drop this compatibility. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20557) Constant Folding: null > (expr) is not constant folded to NULL
Gopal V created HIVE-20557: -- Summary: Constant Folding: null > (expr) is not constant folded to NULL Key: HIVE-20557 URL: https://issues.apache.org/jira/browse/HIVE-20557 Project: Hive Issue Type: Improvement Reporter: Gopal V {code} create temporary table foo(col1 int, col2 int, col3 int, col5 int, col9 int) stored as orc; explain select count(1) from foo where CASE WHEN (col5 is not null) THEN (((col1 / col5) > (col9 / col3))) ELSE ((null > (col9 / col3))) END; explain select count(1), max((col9/col3) > null) from foo where CASE WHEN (col5 is not null) THEN (((col1 / col5) > (col9 / col3))) ELSE (((col9 / col3) > null)) END; {code} There's no sane way to write this query, but this is actually generated by CBO for an expression which looks like {code} case when t_w_firstyear.year_total > 0 then t_w_secyear.year_total / t_w_firstyear.year_total else null end > case when t_s_firstyear.year_total > 0 then t_s_secyear.year_total / t_s_firstyear.year_total else null end {code} -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20509) Plan: fix wasted memory in plans with large partition counts
Gopal V created HIVE-20509: -- Summary: Plan: fix wasted memory in plans with large partition counts Key: HIVE-20509 URL: https://issues.apache.org/jira/browse/HIVE-20509 Project: Hive Issue Type: Bug Components: Query Planning Reporter: Gopal V {code} public void addPathToAlias(Path path, String newAlias){ ArrayList aliases = pathToAliases.get(path); if (aliases == null) { aliases = new ArrayList<>(); StringInternUtils.internUriStringsInPath(path); pathToAliases.put(path, aliases); } aliases.add(newAlias.intern()); } {code} ArrayList::DEFAULT_CAPACITY is 10, so this wastes 500 bytes of memory due to the {{new ArrayList<>();}}. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20501) Vectorization: Closed range fast-path for Fast Long hashset
Gopal V created HIVE-20501: -- Summary: Vectorization: Closed range fast-path for Fast Long hashset Key: HIVE-20501 URL: https://issues.apache.org/jira/browse/HIVE-20501 Project: Hive Issue Type: Improvement Reporter: Gopal V In scenarios where the surrogate keys are entirely contiguous, the cache can offer a fast-path for [min,max], without a further lookup in the hashtable. {code} hive> select min(c_customer_sk), max(c_customer_sk), max(c_customer_sk) - min(c_customer_sk), count(1) from customer; 1 650064996500 {code} -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20490) UDAF: Add an `approx_distinct` to Hive
Gopal V created HIVE-20490: -- Summary: UDAF: Add an `approx_distinct` to Hive Key: HIVE-20490 URL: https://issues.apache.org/jira/browse/HIVE-20490 Project: Hive Issue Type: Improvement Components: UDF Affects Versions: 4.0.0 Reporter: Gopal V This gives a direct pathway to call the equivalent of Hive's default compute_stats() UDAF which computes a HyperLogLog bitVector + approx distinct on a column. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20486) Kafka: Use Row SerDe + vectorization
Gopal V created HIVE-20486: -- Summary: Kafka: Use Row SerDe + vectorization Key: HIVE-20486 URL: https://issues.apache.org/jira/browse/HIVE-20486 Project: Hive Issue Type: Improvement Reporter: Gopal V KafkaHandler returns unvectorized rows which causes the operators downstream to be slower and sub-optimal. Hive has a vectorization shim which allows Kafka streams without complex projections to be wrapped into a vectorized reader via {{hive.vectorized.use.row.serde.deserialize}}. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20455) Log spew from security.authorization.PrivilegeSynchonizer.run
Gopal V created HIVE-20455: -- Summary: Log spew from security.authorization.PrivilegeSynchonizer.run Key: HIVE-20455 URL: https://issues.apache.org/jira/browse/HIVE-20455 Project: Hive Issue Type: Bug Reporter: Gopal V Typo in the class name aside, this keeps running and thrashing through every single DB in the cluster if not all directories are actually accessible to the user who runs HS2. {code} 2018-08-24T03:14:41,932 INFO [PrivilegeSynchonizer] metastore.HiveMetaStore: 1: get_table : tbl=hive.airline_ontime.flights_raw 2018-08-24T03:14:41,932 INFO [PrivilegeSynchonizer] HiveMetaStore.audit: ugi=gopal ip=unknown-ip-addr cmd=get_table : tbl=hive.airline_ontime.flights_raw {code} The minimum fix needed is to move the Thread.sleep() outside of the try/catch block, so that the sleep is triggered even if the attempt throws an exception. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20452) DPP: Simple n-tuple expressions support
Gopal V created HIVE-20452: -- Summary: DPP: Simple n-tuple expressions support Key: HIVE-20452 URL: https://issues.apache.org/jira/browse/HIVE-20452 Project: Hive Issue Type: Improvement Components: Tez Affects Versions: 4.0.0 Reporter: Gopal V {code} create table sales (s_qty bigint, s_item_sk bigint) partitioned by(s_region string, s_date date); insert into sales partition(s_region='USA', s_date='2018-01-01') values (1,1), (2,2); insert into sales partition(s_region='EU', s_date='2018-02-01') values (1,1), (2,2); {code} {code} select count(*) from sales s, agg_sales a where concat(a.s_region, '^', a.s_date) = concat(s.s_region, '^', s.s_date) and s.sold_qty > a.median_s_qty and a.s_item_sk = s.s_item_sk; select count(*) from sales s, agg_sales a where (a.s_region, a.s_date) = (s.s_region, s.s_date) and s.sold_qty > a.median_s_qty and a.s_item_sk = s.s_item_sk; select count(*) from sales s, agg_sales a where a.s_region = s.s_region and a.s_date = s.s_date and s.sold_qty > a.median_s_qty and a.s_item_sk = s.s_item_sk; {code} -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20419) Vectorization: Speed up VectorizationDispatcher.validateInputFormatAndSchemaEvolution() for ACIDv2
Gopal V created HIVE-20419: -- Summary: Vectorization: Speed up VectorizationDispatcher.validateInputFormatAndSchemaEvolution() for ACIDv2 Key: HIVE-20419 URL: https://issues.apache.org/jira/browse/HIVE-20419 Project: Hive Issue Type: Bug Reporter: Gopal V With ACID table, the format and schema layouts are much more strictly controlled - the table cannot be made of partial ORC and partial RCFile. This assumption can remove this loop and the slow check for schema between each partition before vectorizing the operators - the worst-case performance is the common & correct case, where all of them match. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20416) LLAP: Fail to package LLAP if the Xmx computed is 0
Gopal V created HIVE-20416: -- Summary: LLAP: Fail to package LLAP if the Xmx computed is 0 Key: HIVE-20416 URL: https://issues.apache.org/jira/browse/HIVE-20416 Project: Hive Issue Type: Bug Components: llap Reporter: Gopal V {code} if (options.getXmx() != -1) { // Needs more explanation here // Xmx is not the max heap value in JDK8. You need to subtract 50% of the survivor fraction // from this, to get actual usable memory before it goes into GC xmx = options.getXmx(); long xmxMb = (xmx / (1024L * 1024L)); conf.setLong(ConfVars.LLAP_DAEMON_MEMORY_PER_INSTANCE_MB.varname, xmxMb); propsDirectOptions.setProperty(ConfVars.LLAP_DAEMON_MEMORY_PER_INSTANCE_MB.varname, String.valueOf(xmxMb)); {code} needs a check for the value being so low that start-up (or all queries) will fail. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20385) Date: date + int fails to add days
Gopal V created HIVE-20385: -- Summary: Date: date + int fails to add days Key: HIVE-20385 URL: https://issues.apache.org/jira/browse/HIVE-20385 Project: Hive Issue Type: Bug Components: Types Affects Versions: 4.0.0 Reporter: Gopal V {code} select current_date + 5; FAILED: SemanticException [Error 10014]: Line 1:7 Wrong arguments '5': No matching method for class org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPDTIPlus with (date, int) {code} This works in Postgres 9.6 - http://sqlfiddle.com/#!17/9eecb/19253/0 -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20381) Vectorization: Reduce dedup of GroupBy + PTF turns off vectorization
Gopal V created HIVE-20381: -- Summary: Vectorization: Reduce dedup of GroupBy + PTF turns off vectorization Key: HIVE-20381 URL: https://issues.apache.org/jira/browse/HIVE-20381 Project: Hive Issue Type: Bug Components: Vectorization Affects Versions: 3.1.0, 4.0.0 Reporter: Gopal V One of the PTF Reducers in Query51 is not vectorized because there's a reduce deduplication which combines a group-by and a windowing shuffle. {code} | Reducer 8 | | Execution mode: llap | | Reduce Vectorization: | | enabled: true | | enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true | | notVectorizedReason: PTF operator: Only PTF directly under reduce-shuffle is supported | | vectorized: false {code} It vectorizes all PTF vertices (after HIVE-20367), with {{ set hive.optimize.reducededuplication=false;}} -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20362) PTF: Invert sorting for CURRENT_ROW to UNBOUNDED FOLLOWING
Gopal V created HIVE-20362: -- Summary: PTF: Invert sorting for CURRENT_ROW to UNBOUNDED FOLLOWING Key: HIVE-20362 URL: https://issues.apache.org/jira/browse/HIVE-20362 Project: Hive Issue Type: Bug Components: CBO, PTF-Windowing Reporter: Gopal V {code} COALESCE(current_page, FIRST_VALUE(current_page,TRUE) OVER ( PARTITION BY user_id ORDER BY timestamp ROWS BETWEEN CURRENT ROW and UNBOUNDED FOLLOWING)) {code} This is a spilling PTF because it needs an unseen row to generate the current row when executing this. This can be inverted, so that the shuffle sorts it the other way, so the result can be generated from a previously seen row. {code} COALESCE(current_page, LAST_VALUE(current_page,TRUE) OVER ( PARTITION BY user_id ORDER BY timestamp ROWS BETWEEN UNBOUNDED PRECEDING and CURRENT ROW)) {code} This query pattern is bad in general, but seems to exist as a workaround for - HIVE-18145 -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20338) LLAP: Force synthetic file-id for filesystems which have HDFS protocol impls with POSIX mutation semantics
Gopal V created HIVE-20338: -- Summary: LLAP: Force synthetic file-id for filesystems which have HDFS protocol impls with POSIX mutation semantics Key: HIVE-20338 URL: https://issues.apache.org/jira/browse/HIVE-20338 Project: Hive Issue Type: Bug Reporter: Gopal V HDFS protocol does not guarantee immutability of files - the synthetic file-id includes the mtime of the file as well, which is a fail-safe for such filesystems. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20333) CBO: Join removal based on PK-FK declared constraints
Gopal V created HIVE-20333: -- Summary: CBO: Join removal based on PK-FK declared constraints Key: HIVE-20333 URL: https://issues.apache.org/jira/browse/HIVE-20333 Project: Hive Issue Type: Bug Reporter: Gopal V A query of the following shape can have its customer join removed entirely on the basis of the key containment between customer & store_sales. {code} select c_customer_sk,sum(ss_quantity*ss_sales_price) ssales from store_sales ,customer where ss_customer_sk = c_customer_sk group by c_customer_sk; {code} This query after join removal can be encoded in as {code} select ss_customer_sk as c_customer_sk,sum(ss_quantity*ss_sales_price) ssales from store_sales where ss_customer_sk is not null group by ss_customer_sk; {code} The rewrite is not applied today and the current PK-FK relationship does not allow for a nullable relationship (i.e a declared Foreign Key can't be NULL). -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20321) Vectorization: Cut down memory size of 1 col VectorHashKeyWrapper to <1 CacheLine
Gopal V created HIVE-20321: -- Summary: Vectorization: Cut down memory size of 1 col VectorHashKeyWrapper to <1 CacheLine Key: HIVE-20321 URL: https://issues.apache.org/jira/browse/HIVE-20321 Project: Hive Issue Type: Bug Reporter: Gopal V With a full sized LLAP instance, the memory size of the VectorHashKeyWrapper is bigger than the low Xmx JVMs. {code} * 64-bit VM: ** org.apache.hadoop.hive.ql.exec.vector.VectorHashKeyWrapper object internals: OFFSET SIZE TYPE DESCRIPTION VALUE 016 (object header) N/A 16 4 int VectorHashKeyWrapper.hashcodeN/A 20 4 (alignment/padding gap) 24 8 long[] VectorHashKeyWrapper.longValues N/A 32 8 double[] VectorHashKeyWrapper.doubleValuesN/A 40 8 byte[][] VectorHashKeyWrapper.byteValues N/A 48 8 int[] VectorHashKeyWrapper.byteStarts N/A 56 8 int[] VectorHashKeyWrapper.byteLengths N/A 64 8 org.apache.hadoop.hive.serde2.io.HiveDecimalWritable[] VectorHashKeyWrapper.decimalValues N/A 72 8 java.sql.Timestamp[] VectorHashKeyWrapper.timestampValues N/A 80 8 org.apache.hadoop.hive.common.type.HiveIntervalDayTime[] VectorHashKeyWrapper.intervalDayTimeValues N/A 88 8 boolean[] VectorHashKeyWrapper.isNull N/A 96 8 org.apache.hadoop.hive.ql.exec.vector.VectorHashKeyWrapper.HashContext VectorHashKeyWrapper.hashCtx N/A Instance size: 104 bytes Space losses: 4 bytes internal + 0 bytes external = 4 bytes total {code} Pulling this up to a parent class allows for this to be cut down to 32 bytes for the single column case. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20277) Vectorization: Case expressions that return NULL in FILTER
Gopal V created HIVE-20277: -- Summary: Vectorization: Case expressions that return NULL in FILTER Key: HIVE-20277 URL: https://issues.apache.org/jira/browse/HIVE-20277 Project: Hive Issue Type: Bug Reporter: Gopal V In cases like Query89, the vertex with the filter is not vectorized. {code} Filter Operator predicate: CASE WHEN ((avg_window_0 <> 0.0D)) THEN (((abs((_col6 - avg_window_0)) / avg_window_0) > 0.1D)) ELSE (null) END (type: boolean) {code} {code} Reducer 3 Execution mode: llap Reduce Vectorization: enabled: true enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true notVectorizedReason: FILTER operator: Unexpected hive type name void vectorized: false {code} The query specifically has {code} where case when (avg_monthly_sales <> 0) then (abs(sum_sales - avg_monthly_sales) / avg_monthly_sales) else null end > 0.1 {code} while rewriting it to {code} where case when (avg_monthly_sales <> 0) then (abs(sum_sales - avg_monthly_sales) / avg_monthly_sales) > 0.1 else false end {code} does vectorize into {code} Filter Operator Filter Vectorization: className: VectorFilterOperator native: true predicateExpression: SelectColumnIsTrue(col 12:boolean)(children: VectorUDFAdaptor(CASE WHEN ((avg_window_0 <> 0.0D)) THEN (((abs((_col6 - avg_window_0)) / avg_window_0) > 0.1D)) ELSE (false) END)(children: DoubleColNotEqualDoubleScalar(col 7:double, val 0.0) -> 8:boolean, DoubleColGreaterDoubleScalar(col 9:double, val 0.1)(children: DoubleColDivideDoubleColumn(col 10:double, col 7:double)(children: FuncAbsDoubleToDouble(col 9:double)(children: DoubleColSubtractDoubleColumn(col 6:double, col 7:double) -> 9:double) -> 10:double) -> 9:double) -> 11:boolean) -> 12:boolean) predicate: CASE WHEN ((avg_window_0 <> 0.0D)) THEN (((abs((_col6 - avg_window_0)) / avg_window_0) > 0.1D)) ELSE (false) END (type: boolean) Statistics: Num rows: 11 Data size: 5291 Basic stats: COMPLETE Column stats: COMPLETE {code} -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20274) HiveServer2 ObjectInspectorFactory leaks for Struct and List object inspectors
Gopal V created HIVE-20274: -- Summary: HiveServer2 ObjectInspectorFactory leaks for Struct and List object inspectors Key: HIVE-20274 URL: https://issues.apache.org/jira/browse/HIVE-20274 Project: Hive Issue Type: Bug Components: HiveServer2 Affects Versions: 3.0.0, 4.0.0 Reporter: Gopal V Fix in HIVE-19860 needs to be applied to {code} static ConcurrentHashMap cachedStandardListObjectInspector = new ConcurrentHashMap(); ... static ConcurrentHashMap>, StandardStructObjectInspector> cachedStandardStructObjectInspector = new ConcurrentHashMap>, StandardStructObjectInspector>(); ... static ConcurrentHashMap, ColumnarStructObjectInspector> cachedColumnarStructObjectInspector = new ConcurrentHashMap, ColumnarStructObjectInspector>(); {code} And possibly for {code} static ConcurrentHashMap objectInspectorCache = new ConcurrentHashMap(); ... {code} -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20177) Vectorization: Reduce KeyWrapper allocation in GroupBy Streaming mode
Gopal V created HIVE-20177: -- Summary: Vectorization: Reduce KeyWrapper allocation in GroupBy Streaming mode Key: HIVE-20177 URL: https://issues.apache.org/jira/browse/HIVE-20177 Project: Hive Issue Type: Bug Components: Vectorization Reporter: Gopal V The streaming mode for VectorGroupBy allocates a large number of arrays due to VectorKeyHashWrapper::duplicateTo() Since the vectors can't be mutated in-place while a single batch is being processed, this operation can be cut by 1000x by allocating a streaming key at the end of the loop, instead of reallocating within the loop. {code} for(int i = 0; i < batch.size; ++i) { if (!batchKeys[i].equals(streamingKey)) { // We've encountered a new key, must save current one // We can't forward yet, the aggregators have not been evaluated rowsToFlush[flushMark] = currentStreamingAggregators; if (keysToFlush[flushMark] == null) { keysToFlush[flushMark] = (VectorHashKeyWrapper) streamingKey.copyKey(); } else { streamingKey.duplicateTo(keysToFlush[flushMark]); } currentStreamingAggregators = streamAggregationBufferRowPool.getFromPool(); batchKeys[i].duplicateTo(streamingKey); ++flushMark; } {code} The duplicateTo can be pushed out of the loop since there only one to truly keep a copy of is the last unique key in the VRB. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20176) ResultsCache: Time based invalidation is not required for transactional cache entries
Gopal V created HIVE-20176: -- Summary: ResultsCache: Time based invalidation is not required for transactional cache entries Key: HIVE-20176 URL: https://issues.apache.org/jira/browse/HIVE-20176 Project: Hive Issue Type: Bug Affects Versions: 4.0.0 Reporter: Gopal V If a cache entry has explicit transactional validity, it does not need to be invalidated on the timer - the access by a later write-id state will ignore it without having to invalidate it on the timer. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20139) LLAP: execution.mode='only' ignores custom UDTFs
Gopal V created HIVE-20139: -- Summary: LLAP: execution.mode='only' ignores custom UDTFs Key: HIVE-20139 URL: https://issues.apache.org/jira/browse/HIVE-20139 Project: Hive Issue Type: Bug Reporter: Gopal V The LLAP UDF/UDAF check needs to be extended to UDTFs. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20138) Sessions: Closing Tez sessions if opened outside of default sessions throws NPE
Gopal V created HIVE-20138: -- Summary: Sessions: Closing Tez sessions if opened outside of default sessions throws NPE Key: HIVE-20138 URL: https://issues.apache.org/jira/browse/HIVE-20138 Project: Hive Issue Type: Bug Components: Tez Reporter: Gopal V On a busy cluster, if a session is currently in ACCEPTED state (i.e YARN hasn't started it yet), running a query will NPE. {code} java.lang.NullPointerException: null at org.apache.hadoop.hive.ql.exec.tez.TezSessionState.getSession(TezSessionState.java:711) ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT] at org.apache.hadoop.hive.ql.exec.tez.TezSessionState.close(TezSessionState.java:646) ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT] at org.apache.hadoop.hive.ql.exec.tez.TezSessionPoolManager.closeIfNotDefault(TezSessionPoolManager.java:353) ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT] at org.apache.hadoop.hive.ql.exec.tez.TezSessionPoolManager.getSession(TezSessionPoolManager.java:467) ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT] at org.apache.hadoop.hive.ql.exec.tez.WorkloadManagerFederation.getUnmanagedSession(WorkloadManagerFederation.java:66) ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT] at org.apache.hadoop.hive.ql.exec.tez.WorkloadManagerFederation.getSession(WorkloadManagerFederation.java:38) ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT] at org.apache.hadoop.hive.ql.exec.tez.TezTask.execute(TezTask.java:184) ~[hive-exec-4.0.0-SNAPSHOT.jar:4.0.0-SNAPSHOT] {code} -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20113) Shuffle avoidance: Disable 1-1 edges for sorted shuffle
Gopal V created HIVE-20113: -- Summary: Shuffle avoidance: Disable 1-1 edges for sorted shuffle Key: HIVE-20113 URL: https://issues.apache.org/jira/browse/HIVE-20113 Project: Hive Issue Type: Bug Components: Tez Reporter: Gopal V The sorted shuffle avoidance can have some issues when the shuffle data gets broken up into multiple chunks on disk. The 1-1 edge cannot skip the tez final merge - there's no reason for 1-1 to have a final merge at all, it should open a single compressed file and write a single index entry. Until the shuffle issue is resolved & a lot more testing, it is prudent to disable the optimization for sorted shuffle edges and stop rewriting the RS(sorted) = = = RS(sorted) into RS(sorted) = = = RS(FORWARD). -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20103) WM: Only Aggregate DAG counters if at least one is used
Gopal V created HIVE-20103: -- Summary: WM: Only Aggregate DAG counters if at least one is used Key: HIVE-20103 URL: https://issues.apache.org/jira/browse/HIVE-20103 Project: Hive Issue Type: Bug Components: llap Affects Versions: 4.0.0 Reporter: Gopal V {code} status = dagClient.getDAGStatus(EnumSet.of(StatusGetOpts.GET_COUNTERS), checkInterval); TezCounters dagCounters = status.getDAGCounters(); ... if (dagCounters != null && wmContext != null) { Set desiredCounters = wmContext.getSubscribedCounters(); if (desiredCounters != null && !desiredCounters.isEmpty()) { Map currentCounters = getCounterValues(dagCounters, vertexNames, vertexProgressMap, desiredCounters, done); {code} Skip collecting DAG counters unless there at least one desired counter in wmContext. The AM has a hard-lock around the counters, so the current jstacks are full of {code} java.lang.Thread.State: RUNNABLE at java.lang.String.intern(Native Method) at org.apache.hadoop.util.StringInterner.weakIntern(StringInterner.java:71) at org.apache.tez.common.counters.GenericCounter.(GenericCounter.java:50) at org.apache.tez.common.counters.TezCounters$GenericGroup.newCounter(TezCounters.java:65) at org.apache.tez.common.counters.AbstractCounterGroup.addCounterImpl(AbstractCounterGroup.java:92) at org.apache.tez.common.counters.AbstractCounterGroup.findCounter(AbstractCounterGroup.java:104) - locked <0x7efb3ac7af38> (a org.apache.tez.common.counters.TezCounters$GenericGroup) at org.apache.tez.common.counters.AbstractCounterGroup.aggrAllCounters(AbstractCounterGroup.java:204) at org.apache.tez.common.counters.AbstractCounters.aggrAllCounters(AbstractCounters.java:372) - eliminated <0x7efb3ac64ee8> (a org.apache.tez.common.counters.TezCounters) at org.apache.tez.common.counters.AbstractCounters.incrAllCounters(AbstractCounters.java:357) - locked <0x7efb3ac64ee8> (a org.apache.tez.common.counters.TezCounters) at org.apache.tez.dag.app.dag.impl.TaskImpl.getCounters(TaskImpl.java:462) at org.apache.tez.dag.app.dag.impl.VertexImpl.aggrTaskCounters(VertexImpl.java:1342) at org.apache.tez.dag.app.dag.impl.VertexImpl.getAllCounters(VertexImpl.java:1202) at org.apache.tez.dag.app.dag.impl.DAGImpl.aggrTaskCounters(DAGImpl.java:755) at org.apache.tez.dag.app.dag.impl.DAGImpl.getAllCounters(DAGImpl.java:704) at org.apache.tez.dag.app.dag.impl.DAGImpl.getDAGStatus(DAGImpl.java:901) at org.apache.tez.dag.app.dag.impl.DAGImpl.getDAGStatus(DAGImpl.java:940) at org.apache.tez.dag.api.client.DAGClientHandler.getDAGStatus(DAGClientHandler.java:73) {code} -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20101) BloomKFilter: Avoid using the local byte[] arrays entirely
Gopal V created HIVE-20101: -- Summary: BloomKFilter: Avoid using the local byte[] arrays entirely Key: HIVE-20101 URL: https://issues.apache.org/jira/browse/HIVE-20101 Project: Hive Issue Type: Bug Components: Query Processor Affects Versions: 4.0.0 Reporter: Gopal V HIVE-18866 introduced a fast-path for integer -> murmur hash, but the change hasn't been applied to BloomKFilter for integers. {code} public class BloomKFilter { private final byte[] BYTE_ARRAY_4 = new byte[4]; private final byte[] BYTE_ARRAY_8 = new byte[8]; {code} Remove these objects and use the fast-path. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20043) HiveServer2: SessionState has a static sync block around an AtomicBoolean
Gopal V created HIVE-20043: -- Summary: HiveServer2: SessionState has a static sync block around an AtomicBoolean Key: HIVE-20043 URL: https://issues.apache.org/jira/browse/HIVE-20043 Project: Hive Issue Type: Bug Components: HiveServer2 Affects Versions: 3.0.0 Reporter: Gopal V {code} private static void start(SessionState startSs, boolean isAsync, LogHelper console) { ... synchronized(SessionState.class) { if (!startSs.isStarted.compareAndSet(false, true)) { return; } } {code} startSs.isStarted is an AtomicBoolean, which makes it hard to know why this code is locked with a static lock. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20042) HiveServer2: All operations lock on a Single HiveConf object
Gopal V created HIVE-20042: -- Summary: HiveServer2: All operations lock on a Single HiveConf object Key: HIVE-20042 URL: https://issues.apache.org/jira/browse/HIVE-20042 Project: Hive Issue Type: Bug Components: HiveServer2 Affects Versions: 3.0.0 Reporter: Gopal V With the 1000 user test, the session start/tear-down runs only at 100% CPU, which is due to all threads locking on the same HiveConf object. OpenSession locks on 0x0005c091a3a0 {code} "HiveServer2-HttpHandler-Pool: Thread-65084" #65084 prio=5 os_prio=0 tid=0x103bb000 nid=0x4a09 waiting for monitor entry [0x7fc1b0987000] java.lang.Thread.State: BLOCKED (on object monitor) at org.apache.hadoop.conf.Configuration.getOverlay(Configuration.java:1418) - waiting to lock <0x0005c091a3a0> (a org.apache.hadoop.hive.conf.HiveConf) at org.apache.hadoop.conf.Configuration.handleDeprecation(Configuration.java:711) at org.apache.hadoop.conf.Configuration.get(Configuration.java:1437) at org.apache.hadoop.hive.conf.HiveConf.getVar(HiveConf.java:4996) at org.apache.hadoop.hive.conf.HiveConf.getVar(HiveConf.java:5069) at org.apache.hive.service.cli.thrift.ThriftCLIService.getUserName(ThriftCLIService.java:424) at org.apache.hive.service.cli.thrift.ThriftCLIService.getSessionHandle(ThriftCLIService.java:467) at org.apache.hive.service.cli.thrift.ThriftCLIService.OpenSession(ThriftCLIService.java:325) {code} GetOperationStatus locks on the same {code} "HiveServer2-HttpHandler-Pool: Thread-65082" #65082 prio=5 os_prio=0 tid=0x7fc2656be000 nid=0x4a06 waiting for monitor entry [0x7fc3159db000] java.lang.Thread.State: BLOCKED (on object monitor) at org.apache.hadoop.conf.Configuration.getOverlay(Configuration.java:1418) - waiting to lock <0x0005c091a3a0> (a org.apache.hadoop.hive.conf.HiveConf) at org.apache.hadoop.conf.Configuration.handleDeprecation(Configuration.java:711) at org.apache.hadoop.conf.Configuration.get(Configuration.java:1437) at org.apache.hadoop.hive.conf.HiveConf.getVar(HiveConf.java:4996) at org.apache.hadoop.hive.conf.HiveConf.getVar(HiveConf.java:5069) at org.apache.hive.service.cli.thrift.ThriftCLIService.GetOperationStatus(ThriftCLIService.java:709) {code} Session clean up locks on the same {code} "8fd1db09-9f96-49dc-becf-5702826bd4f5 HiveServer2-HttpHandler-Pool: Thread-64981" #64981 prio=5 os_prio=0 tid=0x1d1ab000 nid=0x23d5 waiting for monitor entry [0x7fc1b65e3000] java.lang.Thread.State: BLOCKED (on object monitor) at org.apache.hadoop.conf.Configuration.getOverlay(Configuration.java:1418) - waiting to lock <0x0005c091a3a0> (a org.apache.hadoop.hive.conf.HiveConf) at org.apache.hadoop.conf.Configuration.handleDeprecation(Configuration.java:711) at org.apache.hadoop.conf.Configuration.get(Configuration.java:1177) at org.apache.hadoop.conf.Configuration.getTrimmedStringCollection(Configuration.java:2122) at org.apache.hadoop.hdfs.DFSUtilClient.getNameServiceIds(DFSUtilClient.java:197) at org.apache.hadoop.hdfs.HAUtilClient.isLogicalUri(HAUtilClient.java:53) ... at org.apache.hadoop.hdfs.DistributedFileSystem.delete(DistributedFileSystem.java:959) at org.apache.hadoop.hive.ql.Context.clear(Context.java:724) {code} Hadoop RPC blocks on the same {code} "HiveServer2-HttpHandler-Pool: Thread-59227" #59227 prio=5 os_prio=0 tid=0x7fc270aeb800 nid=0x129b waiting for monitor entry [0x7fc28b7b5000] java.lang.Thread.State: BLOCKED (on object monitor) at org.apache.hadoop.conf.Configuration.getOverlay(Configuration.java:1418) - waiting to lock <0x0005c091a3a0> (a org.apache.hadoop.hive.conf.HiveConf) at org.apache.hadoop.conf.Configuration.handleDeprecation(Configuration.java:711) at org.apache.hadoop.conf.Configuration.get(Configuration.java:1177) at org.apache.hadoop.conf.Configuration.getTrimmed(Configuration.java:1234) at org.apache.hadoop.conf.Configuration.getInt(Configuration.java:1459) at org.apache.hadoop.ipc.Client$Connection.(Client.java:451) at org.apache.hadoop.ipc.Client.getConnection(Client.java:1532) ... at org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1580) at org.apache.hadoop.fs.FileSystem.exists(FileSystem.java:1734) at org.apache.hadoop.fs.FileSystem.deleteOnExit(FileSystem.java:1677) at org.apache.hadoop.hive.ql.session.SessionState.createPath(SessionState.java:791) {code} -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20041) ResultsCache: Improve loggng for concurrent queries
Gopal V created HIVE-20041: -- Summary: ResultsCache: Improve loggng for concurrent queries Key: HIVE-20041 URL: https://issues.apache.org/jira/browse/HIVE-20041 Project: Hive Issue Type: Improvement Components: Diagnosability Reporter: Gopal V The logging for QueryResultsCache ends up printing information without context, like {code} 2018-06-30T17:48:45,502 INFO [HiveServer2-Background-Pool: Thread-166] results.QueryResultsCache: Waiting on pending cacheEntry {code} {code} 2018-06-30T17:50:17,963 INFO [HiveServer2-Background-Pool: Thread-145] ql.Driver: savedToCache: true {code} The previous lines for this are in DEBUG level, so the logging ends up being useless at INFO level to debug. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20040) JDBC: HTTP listen queue is 50 and SYNs are lost
Gopal V created HIVE-20040: -- Summary: JDBC: HTTP listen queue is 50 and SYNs are lost Key: HIVE-20040 URL: https://issues.apache.org/jira/browse/HIVE-20040 Project: Hive Issue Type: Bug Reporter: Gopal V When testing with 5000 concurrent users, the JDBC HTTP port ends up overflowing on SYNs when the HS2 gc pauses. This is because each getQueryProgress request is an independent HTTP request, so unlike the BINARY mode, there are more connections being established & closed in HTTP mode. {code} LISTEN 0 50 *:10004*:* {code} This turns into connection errors when enabling {{net.ipv4.tcp_abort_on_overflow=1}}, but the better approach is to enqueue the connections until the HS2 is done with its GC pause. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-20021) LLAP: Fall back to Synthetic File-ids when getting a HdfsConstants.GRANDFATHER_INODE_ID
Gopal V created HIVE-20021: -- Summary: LLAP: Fall back to Synthetic File-ids when getting a HdfsConstants.GRANDFATHER_INODE_ID Key: HIVE-20021 URL: https://issues.apache.org/jira/browse/HIVE-20021 Project: Hive Issue Type: Bug Reporter: Gopal V HDFS client implementations have multiple server implementations, which do not all support the inodes for file locations. If the client returns a 0 InodeId, fall back to the synthetic ones. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-19985) ACID: Skip decoding the ROW__ID sections for read-only queries
Gopal V created HIVE-19985: -- Summary: ACID: Skip decoding the ROW__ID sections for read-only queries Key: HIVE-19985 URL: https://issues.apache.org/jira/browse/HIVE-19985 Project: Hive Issue Type: Improvement Components: Transactions Reporter: Gopal V For a base_n file there are no aborted transactions within the file and if there are no pending delete deltas, the entire ACID ROW__ID can be skipped for all read-only queries (i.e SELECT), though it still needs to be projected out for MERGE, UPDATE and DELETE queries. This patch tries to entirely ignore the ACID ROW__ID fields for all tables where there are no possible deletes or aborted transactions for an ACID split. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-19955) ACID: Pre-filter the delete event registry using insert delta ranges
Gopal V created HIVE-19955: -- Summary: ACID: Pre-filter the delete event registry using insert delta ranges Key: HIVE-19955 URL: https://issues.apache.org/jira/browse/HIVE-19955 Project: Hive Issue Type: Bug Reporter: Gopal V Since the delete deltas that will be used for the ACID impl is limited to the txn range encoded within the insert deltas, it is not useful to load any delete events for any row outside of the current file's range. If the insert delta has "delta_3_3_0", then the "writeid=3" can be applied to the delete delta list while loading it into memory - if the file has "base_3", then the filter becomes "writeid <= 3". -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-19890) ACID: Inherit bucket-id from original ROW_ID for delete deltas
Gopal V created HIVE-19890: -- Summary: ACID: Inherit bucket-id from original ROW_ID for delete deltas Key: HIVE-19890 URL: https://issues.apache.org/jira/browse/HIVE-19890 Project: Hive Issue Type: Bug Components: Transactions Affects Versions: 3.0.0 Reporter: Gopal V The ACID delete deltas for unbucketed tables are written to arbitrary files, which should instead be shuffled using the bucket-id instead of hash(ROW__ID). -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-19790) Metastore upgrade: 3.1.0 upgrade script is slow and non-idempotent
Gopal V created HIVE-19790: -- Summary: Metastore upgrade: 3.1.0 upgrade script is slow and non-idempotent Key: HIVE-19790 URL: https://issues.apache.org/jira/browse/HIVE-19790 Project: Hive Issue Type: Bug Components: Standalone Metastore Reporter: Gopal V Because of the giant bit-vectors stored on mysql, the update of PART_COL_STATS is very slow and also is not idempotent. {code} -- UPDATE `PART_COL_STATS` SET `CAT_NAME` = 'hive' -- Query OK, 0 rows affected (4 min 1.57 sec) Rows matched: 778025 Changed: 0 Warnings: 0 {code} Adding a filter speeds it up because it will no longer overwrite {code} mysql> explain UPDATE `PART_COL_STATS` SET `CAT_NAME` = 'hive' where `CAT_NAME` =''; -- explain UPDATE `PART_COL_STATS` SET `CAT_NAME` = 'hive' where `CAT_NAME` ='' -- ++-++---+---+---+-+---+--+--+ | id | select_type | table | type | possible_keys | key | key_len | ref | rows | Extra| ++-++---+---+---+-+---+--+--+ | 1 | SIMPLE | PART_COL_STATS | range | PCS_STATS_IDX | PCS_STATS_IDX | 258 | const |1 | Using where; Using temporary | ++-++---+---+---+-+---+--+--+ 1 row in set (0.00 sec) {code} this would be much faster to re-run and would not accidentally overwrite any existing CAT_NAMEs. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-19446) QueryCache: Transaction lists needed for pending cache entries
Gopal V created HIVE-19446: -- Summary: QueryCache: Transaction lists needed for pending cache entries Key: HIVE-19446 URL: https://issues.apache.org/jira/browse/HIVE-19446 Project: Hive Issue Type: Bug Reporter: Gopal V Hive query-cache needs a transactional list, even when the entry is pending state so that other identical queries with the same transactional state can wait for the first query to complete, instead of triggering their own instance. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-19376) Statistics: switch to 10bit HLL by default for Hive
Gopal V created HIVE-19376: -- Summary: Statistics: switch to 10bit HLL by default for Hive Key: HIVE-19376 URL: https://issues.apache.org/jira/browse/HIVE-19376 Project: Hive Issue Type: Sub-task Components: Statistics Reporter: Gopal V This reduces the memory usage for the metastore cache and the size of bit-vectors in the DB by 16x. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-19369) Locks: Add new lock implementations for always zero-wait readers
Gopal V created HIVE-19369: -- Summary: Locks: Add new lock implementations for always zero-wait readers Key: HIVE-19369 URL: https://issues.apache.org/jira/browse/HIVE-19369 Project: Hive Issue Type: Improvement Reporter: Gopal V Hive Locking with Micro-managed and full-ACID tables needs a better locking implementation which allows for no-wait readers always. EXCL_DROP EXCL_WRITE SHARED_WRITE SHARED_READ Short write-up EXCL_DROP is a "drop partition" or "drop table" and waits for all others to exit EXCL_WRITE excludes all writes and will wait for all existing SHARED_WRITE to exit. SHARED_WRITE allows all SHARED_WRITES to go through, but will wait for an EXCL_WRITE & EXCL_DROP (waiting so that you can do drop + insert in different threads). SHARED_READ does not wait for any lock - it fails fast for a pending EXCL_DROP, because even if there is an EXCL_WRITE or SHARED_WRITE pending, there's no semantic reason to wait for them to succeed before going ahead with a SHARED_WRITE. a select * => SHARED_READ an insert into => SHARED_WRITE an insert overwrite or MERGE => EXCL_WRITE a drop table => EXCL_DROP TODO: The fate of the compactor needs to be added to this before it is a complete description. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-19368) Metastore: log a warning with table-name + partition-count when get_partitions returns >10k partitions
Gopal V created HIVE-19368: -- Summary: Metastore: log a warning with table-name + partition-count when get_partitions returns >10k partitions Key: HIVE-19368 URL: https://issues.apache.org/jira/browse/HIVE-19368 Project: Hive Issue Type: Improvement Reporter: Gopal V Ran into this particular letter from the trenches & would like a normal WARN log for it. https://www.slideshare.net/Hadoop_Summit/hive-at-yahoo-letters-from-the-trenches/24 -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-19360) CBO: Add an "optimizedSQL" to QueryPlan object
Gopal V created HIVE-19360: -- Summary: CBO: Add an "optimizedSQL" to QueryPlan object Key: HIVE-19360 URL: https://issues.apache.org/jira/browse/HIVE-19360 Project: Hive Issue Type: Improvement Components: CBO, Diagnosability Affects Versions: 3.1.0 Reporter: Gopal V Calcite RelNodes can be converted back into SQL (as the new JDBC storage handler does), which allows Hive to print out the post CBO plan as a SQL query instead of having to guess the join orders from the subsequent Tez plan. The query generated might not be always valid SQL at this point, but is a world ahead of DAG plans in readability. Eg. tpc-ds Query4 CTEs gets expanded to {code} SELECT t16.$f3 customer_preferred_cust_flag FROM (SELECT t0.c_customer_id $f0, SUM((t2.ws_ext_list_price - t2.ws_ext_wholesale_cost - t2.ws_ext_discount_amt + t2.ws_ext_sales_price) / CAST(2 AS DECIMAL(10, 0))) $f8 FROM (SELECT c_customer_sk, c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address FROM default.customer WHERE c_customer_sk IS NOT NULL AND c_customer_id IS NOT NULL) t0 INNER JOIN ( (SELECT ws_sold_date_sk, ws_bill_customer_sk, ws_ext_discount_amt, ws_ext_sales_price, ws_ext_wholesale_cost, ws_ext_list_price FROM default.web_sales WHERE ws_bill_customer_sk IS NOT NULL AND ws_sold_date_sk IS NOT NULL) t2 INNER JOIN (SELECT d_date_sk, CAST(2002 AS INTEGER) d_year FROM default.date_dim WHERE d_year = 2002 AND d_date_sk IS NOT NULL) t4 ON t2.ws_sold_date_sk = t4.d_date_sk) ON t0.c_customer_sk = t2.ws_bill_customer_sk GROUP BY t0.c_customer_id, t0.c_first_name, t0.c_last_name, t0.c_preferred_cust_flag, t0.c_birth_country, t0.c_login, t0.c_email_address) t7 INNER JOIN ( (SELECT t9.c_customer_id $f0, t9.c_preferred_cust_flag $f3, SUM((t11.ss_ext_list_price - t11.ss_ext_wholesale_cost - t11.ss_ext_discount_amt + t11.ss_ext_sales_price) / CAST(2 AS DECIMAL(10, 0))) $f8 FROM (SELECT c_customer_sk, c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, c_login, c_email_address FROM default.customer WHERE c_customer_sk IS NOT NULL AND c_customer_id IS NOT NULL) t9 INNER JOIN ( (SELECT ss_sold_date_sk, ss_customer_sk, ss_ext_discount_amt, ss_ext_sales_price, ss_ext_wholesale_cost, ss_ext_list_price FROM default.store_sales WHERE ss_customer_sk IS NOT NULL AND ss_sold_date_sk IS NOT NULL) t11 INNER JOIN (SELECT d_date_sk, CAST(2002 AS INTEGER) d_year FROM default.date_dim WHERE d_year = 2002 AND d_date_sk IS NOT NULL) t13 ON t11.ss_sold_date_sk = t13.d_date_sk) ON t9.c_customer_sk = t11.ss_customer_sk GROUP BY t9.c_customer_id, t9.c_first_name, t9.c_last_name, t9.c_preferred_cust_flag, t9.c_birth_country, t9.c_login, t9.c_email_address) t16 INNER JOIN ( (SELECT t18.c_customer_id $f0, SUM((t20.cs_ext_list_price - t20.cs_ext_wholesale_cost - t20.cs_ext_discount_amt + t20.cs_ext_sales_price) / CAST(2 AS DECIMAL(10, 0))) $f8 FROM (SELECT c_customer_sk, c_customer_id, c_first_name, c_last_name, c_preferred_cust_
[jira] [Created] (HIVE-19290) Statistics: Timestamp statistics support
Gopal V created HIVE-19290: -- Summary: Statistics: Timestamp statistics support Key: HIVE-19290 URL: https://issues.apache.org/jira/browse/HIVE-19290 Project: Hive Issue Type: New Feature Components: Standalone Metastore, Statistics Reporter: Gopal V https://github.com/apache/hive/blob/master/standalone-metastore/src/main/thrift/hive_metastore.thrift#L533 has no support for Timestamp as a statistics type. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-19247) StatsOptimizer: Missing stats fast-path for Date/Timestamp
Gopal V created HIVE-19247: -- Summary: StatsOptimizer: Missing stats fast-path for Date/Timestamp Key: HIVE-19247 URL: https://issues.apache.org/jira/browse/HIVE-19247 Project: Hive Issue Type: Bug Components: Statistics Affects Versions: 2.3.2, 2.2.0, 3.0.0 Reporter: Gopal V {code} 2018-04-19T18:57:24,268 DEBUG [67259108-c184-4c92-9e18-9e296922 HiveServer2-Handler-Pool: Thread-73]: optimizer.StatsOptimizer (StatsOptimizer.java:process(614)) - Unsupported type: date encountered in metadata optimizer for column : jour {code} {code} if (udaf instanceof GenericUDAFMin) { ExprNodeColumnDesc colDesc = (ExprNodeColumnDesc)exprMap.get(((ExprNodeColumnDesc)aggr.getParameters().get(0)).getColumn()); String colName = colDesc.getColumn(); StatType type = getType(colDesc.getTypeString()); if (!tbl.isPartitioned()) { if (!StatsSetupConst.areColumnStatsUptoDate(tbl.getParameters(), colName)) { Logger.debug("Stats for table : " + tbl.getTableName() + " column " + colName + " are not up to date."); return null; } ColumnStatisticsData statData = hive.getMSC().getTableColumnStatistics( tbl.getDbName(), tbl.getTableName(), Lists.newArrayList(colName)) .get(0).getStatsData(); String name = colDesc.getTypeString().toUpperCase(); switch (type) { case Integeral: { LongSubType subType = LongSubType.valueOf(name); LongColumnStatsData lstats = statData.getLongStats(); if (lstats.isSetLowValue()) { oneRow.add(subType.cast(lstats.getLowValue())); } else { oneRow.add(null); } break; } case Double: { DoubleSubType subType = DoubleSubType.valueOf(name); DoubleColumnStatsData dstats = statData.getDoubleStats(); if (dstats.isSetLowValue()) { oneRow.add(subType.cast(dstats.getLowValue())); } else { oneRow.add(null); } break; } default: // unsupported type Logger.debug("Unsupported type: " + colDesc.getTypeString() + " encountered in " + "metadata optimizer for column : " + colName); return null; } } {code} {code} enum StatType{ Integeral, Double, String, Boolean, Binary, Unsupported } enum LongSubType { BIGINT { @Override Object cast(long longValue) { return longValue; } }, INT { @Override Object cast(long longValue) { return (int)longValue; } }, SMALLINT { @Override Object cast(long longValue) { return (short)longValue; } }, TINYINT { @Override Object cast(long longValue) { return (byte)longValue; } }; abstract Object cast(long longValue); } {code} Date/Timestamp are stored as Integral stats (& also the typo there). -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-19213) QueryCache: Stop opening transactions on _dummy_table
Gopal V created HIVE-19213: -- Summary: QueryCache: Stop opening transactions on _dummy_table Key: HIVE-19213 URL: https://issues.apache.org/jira/browse/HIVE-19213 Project: Hive Issue Type: Bug Reporter: Gopal V When the query-cache is hit, the ACID TxnManager opens a txn + lock + heartbeats on _dummy_database._dummy_table. This is blocking the query-cache from jumping through the fast-paths for transactions. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Created] (HIVE-19199) ACID: DbTxnManager heartbeat-service needs static sync init
Gopal V created HIVE-19199: -- Summary: ACID: DbTxnManager heartbeat-service needs static sync init Key: HIVE-19199 URL: https://issues.apache.org/jira/browse/HIVE-19199 Project: Hive Issue Type: Bug Reporter: Gopal V {code} private synchronized void initHeartbeatExecutorService() { if (heartbeatExecutorService != null && !heartbeatExecutorService.isShutdown() && !heartbeatExecutorService.isTerminated()) { return; } ... {code} heartbeatExecutorService is a static field, so this needs class sync, not object sync. -- This message was sent by Atlassian JIRA (v7.6.3#76005)